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/08/13 04:28:58 UTC

svn commit: r1617652 [1/7] - in /hive/branches/cbo: ./ ant/ ant/src/org/apache/hadoop/hive/ant/ beeline/ beeline/src/java/org/apache/hive/beeline/ beeline/src/main/resources/ cli/src/java/org/apache/hadoop/hive/cli/ common/src/java/org/apache/hadoop/hi...

Author: gunther
Date: Wed Aug 13 02:28:54 2014
New Revision: 1617652

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

Added:
    hive/branches/cbo/itests/hive-unit-hadoop2/src/test/java/org/apache/hadoop/hive/ql/security/TestPasswordWithCredentialProvider.java
      - copied unchanged from r1617650, hive/trunk/itests/hive-unit-hadoop2/src/test/java/org/apache/hadoop/hive/ql/security/TestPasswordWithCredentialProvider.java
    hive/branches/cbo/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestPasswordWithConfig.java
      - copied unchanged from r1617650, hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestPasswordWithConfig.java
    hive/branches/cbo/itests/src/
      - copied from r1617650, hive/trunk/itests/src/
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/processors/ListResourceProcessor.java
      - copied unchanged from r1617650, hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/processors/ListResourceProcessor.java
    hive/branches/cbo/ql/src/test/queries/clientpositive/alter_merge_3.q
      - copied unchanged from r1617650, hive/trunk/ql/src/test/queries/clientpositive/alter_merge_3.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/alter_rename_table.q
      - copied unchanged from r1617650, hive/trunk/ql/src/test/queries/clientpositive/alter_rename_table.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/bucket6.q
      - copied unchanged from r1617650, hive/trunk/ql/src/test/queries/clientpositive/bucket6.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/vector_data_types.q
      - copied unchanged from r1617650, hive/trunk/ql/src/test/queries/clientpositive/vector_data_types.q
    hive/branches/cbo/ql/src/test/results/clientpositive/alter_merge_3.q.out
      - copied unchanged from r1617650, hive/trunk/ql/src/test/results/clientpositive/alter_merge_3.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/alter_rename_table.q.out
      - copied unchanged from r1617650, hive/trunk/ql/src/test/results/clientpositive/alter_rename_table.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/bucket6.q.out
      - copied unchanged from r1617650, hive/trunk/ql/src/test/results/clientpositive/bucket6.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/tez/vector_data_types.q.out
      - copied unchanged from r1617650, hive/trunk/ql/src/test/results/clientpositive/tez/vector_data_types.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/vector_data_types.q.out
      - copied unchanged from r1617650, hive/trunk/ql/src/test/results/clientpositive/vector_data_types.q.out
Modified:
    hive/branches/cbo/   (props changed)
    hive/branches/cbo/ant/pom.xml
    hive/branches/cbo/ant/src/org/apache/hadoop/hive/ant/QTestGenTask.java
    hive/branches/cbo/beeline/pom.xml
    hive/branches/cbo/beeline/src/java/org/apache/hive/beeline/BeeLine.java
    hive/branches/cbo/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java
    hive/branches/cbo/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java
    hive/branches/cbo/beeline/src/java/org/apache/hive/beeline/SeparatedValuesOutputFormat.java
    hive/branches/cbo/beeline/src/main/resources/BeeLine.properties
    hive/branches/cbo/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java
    hive/branches/cbo/common/src/java/org/apache/hadoop/hive/common/FileUtils.java
    hive/branches/cbo/common/src/java/org/apache/hadoop/hive/common/JavaUtils.java
    hive/branches/cbo/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
    hive/branches/cbo/conf/hive-default.xml.template
    hive/branches/cbo/data/conf/hive-site.xml
    hive/branches/cbo/data/conf/tez/hive-site.xml
    hive/branches/cbo/data/scripts/q_test_cleanup.sql
    hive/branches/cbo/data/scripts/q_test_init.sql
    hive/branches/cbo/hbase-handler/src/test/templates/TestHBaseCliDriver.vm
    hive/branches/cbo/hbase-handler/src/test/templates/TestHBaseNegativeCliDriver.vm
    hive/branches/cbo/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/SpecialCases.java
    hive/branches/cbo/hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hive/hcatalog/pig/HCatLoader.java
    hive/branches/cbo/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestLocationQueries.java
    hive/branches/cbo/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java
    hive/branches/cbo/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java
    hive/branches/cbo/itests/qtest/pom.xml
    hive/branches/cbo/itests/src/test/resources/testconfiguration.properties
    hive/branches/cbo/itests/util/src/main/java/org/apache/hadoop/hive/hbase/HBaseQTestUtil.java
    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/CheckColumnAccessHook.java
    hive/branches/cbo/metastore/src/java/org/apache/hadoop/hive/metastore/HiveAlterHandler.java
    hive/branches/cbo/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
    hive/branches/cbo/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
    hive/branches/cbo/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
    hive/branches/cbo/metastore/src/java/org/apache/hadoop/hive/metastore/Warehouse.java
    hive/branches/cbo/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.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/Driver.java
    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/HashTableSinkOperator.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/persistence/BytesBytesMultiHashMap.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HashMapWrapper.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinBytesTableContainer.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HashTableLoader.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedBatchUtil.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedColumnarSerDe.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatchCtx.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ConstantVectorExpression.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpressionWriterFactory.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateString.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/index/AggregateIndexHandler.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/index/TableBasedIndexHandler.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.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/optimizer/AbstractBucketJoinProc.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/AbstractSMBJoinProc.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketingSortingReduceSinkOptimizer.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagate.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GroupByOptimizer.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/IndexUtils.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/index/RewriteCanApplyCtx.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/index/RewriteCanApplyProcFactory.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/index/RewriteGBUsingIndex.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/index/RewriteQueryUsingAggregateIndex.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/index/RewriteQueryUsingAggregateIndexCtx.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/NullScanOptimizer.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/index/IndexWhereProcessor.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/index/IndexWhereTaskDispatcher.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ppr/PartitionPruner.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnAccessAnalyzer.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnAccessInfo.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/parse/IndexUpdater.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseContext.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/parse/authorization/HiveAuthorizationTaskFactoryImpl.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterIndexDesc.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableAlterPartDesc.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableSimpleDesc.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/plan/MapJoinDesc.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/plan/PrivilegeObjectDesc.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/plan/RenamePartitionDesc.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowColumnsDesc.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowGrantDesc.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/plan/Statistics.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandProcessor.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandProcessorFactory.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/processors/HiveCommand.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationUtils.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HivePrivilegeObject.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveV1Authorizer.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFAverage.java
    hive/branches/cbo/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java
    hive/branches/cbo/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizedRowBatchCtx.java
    hive/branches/cbo/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHive.java
    hive/branches/cbo/ql/src/test/org/apache/hadoop/hive/ql/parse/TestQBCompact.java
    hive/branches/cbo/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/PrivilegesTestBase.java
    hive/branches/cbo/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestHiveAuthorizationTaskFactory.java
    hive/branches/cbo/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestPrivilegesV1.java
    hive/branches/cbo/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/TestPrivilegesV2.java
    hive/branches/cbo/ql/src/test/queries/clientpositive/having.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/mapjoin_mapjoin.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/vector_coalesce.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/vectorized_date_funcs.q
    hive/branches/cbo/ql/src/test/results/clientnegative/alter_concatenate_indexed_table.q.out
    hive/branches/cbo/ql/src/test/results/clientnegative/alter_view_failure6.q.out
    hive/branches/cbo/ql/src/test/results/clientnegative/merge_negative_1.q.out
    hive/branches/cbo/ql/src/test/results/clientnegative/merge_negative_2.q.out
    hive/branches/cbo/ql/src/test/results/clientnegative/show_columns3.q.out
    hive/branches/cbo/ql/src/test/results/clientnegative/show_tableproperties1.q.out
    hive/branches/cbo/ql/src/test/results/clientnegative/temp_table_index.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/annotate_stats_join.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/drop_multi_partitions.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/having.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/input3.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/insert2_overwrite_partitions.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/mapjoin_mapjoin.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/ql_rewrite_gbtoidx.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/show_create_table_db_table.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/show_tblproperties.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/temp_table_names.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/temp_table_precedence.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/tez/having.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/tez/mapjoin_mapjoin.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/tez/vector_cast_constant.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/tez/vectorization_14.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/tez/vectorization_15.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/tez/vectorization_9.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/tez/vectorization_short_regress.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/union20.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/vector_between_in.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/vector_cast_constant.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/vector_coalesce.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/vector_decimal_mapjoin.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/vector_decimal_math_funcs.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/vector_elt.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/vectorization_14.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/vectorization_15.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/vectorization_16.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/vectorization_9.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/vectorization_div0.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/vectorization_short_regress.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/vectorized_date_funcs.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/vectorized_math_funcs.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/vectorized_parquet.q.out
    hive/branches/cbo/ql/src/test/templates/TestCliDriver.vm
    hive/branches/cbo/ql/src/test/templates/TestCompareCliDriver.vm
    hive/branches/cbo/ql/src/test/templates/TestNegativeCliDriver.vm
    hive/branches/cbo/ql/src/test/templates/TestParse.vm
    hive/branches/cbo/ql/src/test/templates/TestParseNegative.vm
    hive/branches/cbo/service/src/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java
    hive/branches/cbo/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java
    hive/branches/cbo/shims/0.20/src/main/java/org/apache/hadoop/hive/shims/Hadoop20Shims.java
    hive/branches/cbo/shims/0.20S/src/main/java/org/apache/hadoop/hive/shims/Hadoop20SShims.java
    hive/branches/cbo/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
    hive/branches/cbo/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShims.java

Propchange: hive/branches/cbo/
------------------------------------------------------------------------------
  Merged /hive/trunk:r1616376-1617650

Modified: hive/branches/cbo/ant/pom.xml
URL: http://svn.apache.org/viewvc/hive/branches/cbo/ant/pom.xml?rev=1617652&r1=1617651&r2=1617652&view=diff
==============================================================================
--- hive/branches/cbo/ant/pom.xml (original)
+++ hive/branches/cbo/ant/pom.xml Wed Aug 13 02:28:54 2014
@@ -39,6 +39,11 @@
       <artifactId>commons-lang</artifactId>
       <version>${commons-lang.version}</version>
     </dependency>
+      <dependency>
+        <groupId>com.google.guava</groupId>
+        <artifactId>guava</artifactId>
+        <version>${guava.version}</version>
+      </dependency>
     <dependency>
       <groupId>org.apache.ant</groupId>
       <artifactId>ant</artifactId>

Modified: hive/branches/cbo/ant/src/org/apache/hadoop/hive/ant/QTestGenTask.java
URL: http://svn.apache.org/viewvc/hive/branches/cbo/ant/src/org/apache/hadoop/hive/ant/QTestGenTask.java?rev=1617652&r1=1617651&r2=1617652&view=diff
==============================================================================
--- hive/branches/cbo/ant/src/org/apache/hadoop/hive/ant/QTestGenTask.java (original)
+++ hive/branches/cbo/ant/src/org/apache/hadoop/hive/ant/QTestGenTask.java Wed Aug 13 02:28:54 2014
@@ -32,10 +32,11 @@ import java.util.ArrayList;
 import java.util.regex.Pattern;
 import java.util.HashMap;
 
+import com.google.common.base.Splitter;
+import com.google.common.collect.Sets;
 import org.apache.commons.lang.StringUtils;
 import org.apache.tools.ant.BuildException;
 import org.apache.tools.ant.Task;
-
 import org.apache.velocity.app.VelocityEngine;
 import org.apache.velocity.Template;
 import org.apache.velocity.VelocityContext;
@@ -45,6 +46,9 @@ import org.apache.velocity.exception.Res
 import org.apache.velocity.runtime.RuntimeConstants;
 
 public class QTestGenTask extends Task {
+   private static final Splitter CSV_SPLITTER = Splitter.on(',')
+       .trimResults()
+       .omitEmptyStrings();
 
   public class IncludeFilter implements FileFilter {
 
@@ -74,9 +78,8 @@ public class QTestGenTask extends Task {
       }
       return true;
     }
-    
   }
-  
+
   public class DisabledQFileFilter extends IncludeFilter {
     public DisabledQFileFilter(Set<String> includeOnly) {
       super(includeOnly);
@@ -87,17 +90,16 @@ public class QTestGenTask extends Task {
         return false;
       }
       return !fpath.isDirectory() && fpath.getName().endsWith(".q.disabled");
-    }  
+    }
   }
-  
+
   public class QFileRegexFilter extends QFileFilter {
     Pattern filterPattern;
-    
     public QFileRegexFilter(String filter, Set<String> includeOnly) {
       super(includeOnly);
       filterPattern = Pattern.compile(filter);
     }
-    
+
     public boolean accept(File filePath) {
       if (!super.accept(filePath)) {
         return false;
@@ -110,17 +112,17 @@ public class QTestGenTask extends Task {
   private List<String> templatePaths = new ArrayList<String>();
 
   private String hiveRootDirectory;
-  
+
   private String outputDirectory;
- 
+
   private String queryDirectory;
- 
+
   private String queryFile;
 
   private String includeQueryFile;
 
   private String excludeQueryFile;
-  
+
   private String queryFileRegex;
 
   private String resultsDirectory;
@@ -138,9 +140,13 @@ public class QTestGenTask extends Task {
   private String hiveConfDir;
 
   private String runDisabled;
-  
+
   private String hadoopVersion;
 
+  private String initScript;
+
+  private String cleanupScript;
+
   public void setHadoopVersion(String ver) {
     this.hadoopVersion = ver;
   }
@@ -156,7 +162,7 @@ public class QTestGenTask extends Task {
   public String getHiveConfDir() {
     return hiveConfDir;
   }
-  
+
   public void setClusterMode(String clusterMode) {
     this.clusterMode = clusterMode;
   }
@@ -197,6 +203,22 @@ public class QTestGenTask extends Task {
     return template;
   }
 
+  public String getInitScript() {
+    return initScript;
+  }
+
+  public void setInitScript(String initScript) {
+    this.initScript = initScript;
+  }
+
+  public String getCleanupScript() {
+    return cleanupScript;
+  }
+
+  public void setCleanupScript(String cleanupScript) {
+    this.cleanupScript = cleanupScript;
+  }
+
   public void setHiveRootDirectory(File hiveRootDirectory) {
     try {
       this.hiveRootDirectory = hiveRootDirectory.getCanonicalPath();
@@ -208,10 +230,10 @@ public class QTestGenTask extends Task {
   public String getHiveRootDirectory() {
     return hiveRootDirectory;
   }
-  
+
   public void setTemplatePath(String templatePath) throws Exception {
     templatePaths.clear();
-    for (String relativePath : templatePath.split(",")) {
+    for (String relativePath : CSV_SPLITTER.split(templatePath)) {
       templatePaths.add(project.resolveFile(relativePath).getCanonicalPath());
     }
     System.out.println("Template Path:" + getTemplatePath());
@@ -316,7 +338,7 @@ public class QTestGenTask extends Task {
 
     Set<String> includeOnly = null;
     if (includeQueryFile != null && !includeQueryFile.isEmpty()) {
-      includeOnly = new HashSet<String>(Arrays.asList(includeQueryFile.split(",")));
+      includeOnly = Sets.<String>newHashSet(CSV_SPLITTER.split(includeQueryFile));
     }
 
     List<File> qFiles;
@@ -326,7 +348,7 @@ public class QTestGenTask extends Task {
     File outDir = null;
     File resultsDir = null;
     File logDir = null;
-    
+
     try {
       // queryDirectory should not be null
       queryDir = new File(queryDirectory);
@@ -335,7 +357,7 @@ public class QTestGenTask extends Task {
       Set<File> testFiles = new HashSet<File>();
       if (queryFile != null && !queryFile.equals("")) {
         // The user may have passed a list of files - comma separated
-        for (String qFile : queryFile.split(",")) {
+        for (String qFile : CSV_SPLITTER.split(queryFile)) {
           if (includeOnly != null && !includeOnly.contains(qFile)) {
             continue;
           }
@@ -346,7 +368,7 @@ public class QTestGenTask extends Task {
           }
         }
       } else if (queryFileRegex != null && !queryFileRegex.equals("")) {
-        for (String regex : queryFileRegex.split(",")) {
+        for (String regex : CSV_SPLITTER.split(queryFileRegex)) {
           testFiles.addAll(Arrays.asList(queryDir.listFiles(
               new QFileRegexFilter(regex, includeOnly))));
         }
@@ -358,7 +380,7 @@ public class QTestGenTask extends Task {
 
       if (excludeQueryFile != null && !excludeQueryFile.equals("")) {
         // Exclude specified query files, comma separated
-        for (String qFile : excludeQueryFile.split(",")) {
+        for (String qFile : CSV_SPLITTER.split(excludeQueryFile)) {
           if (null != queryDir) {
             testFiles.remove(new File(queryDir, qFile));
           } else {
@@ -444,6 +466,8 @@ public class QTestGenTask extends Task {
       ctx.put("clusterMode", clusterMode);
       ctx.put("hiveConfDir", escapePath(hiveConfDir));
       ctx.put("hadoopVersion", hadoopVersion);
+      ctx.put("initScript", initScript);
+      ctx.put("cleanupScript", cleanupScript);
 
       File outFile = new File(outDir, className + ".java");
       FileWriter writer = new FileWriter(outFile);
@@ -466,9 +490,11 @@ public class QTestGenTask extends Task {
       throw new BuildException("Generation failed", e);
     }
   }
+
   private String relativePath(File hiveRootDir, File file) {
     return escapePath(hiveRootDir.toURI().relativize(file.toURI()).getPath());
-  }  
+  }
+
   private static String escapePath(String path) {
     if (System.getProperty("os.name").toLowerCase().startsWith("win")) {
       // Escape the backward slash in CanonicalPath if the unit test runs on windows

Modified: hive/branches/cbo/beeline/pom.xml
URL: http://svn.apache.org/viewvc/hive/branches/cbo/beeline/pom.xml?rev=1617652&r1=1617651&r2=1617652&view=diff
==============================================================================
--- hive/branches/cbo/beeline/pom.xml (original)
+++ hive/branches/cbo/beeline/pom.xml Wed Aug 13 02:28:54 2014
@@ -48,7 +48,6 @@
       <groupId>org.apache.hive</groupId>
       <artifactId>hive-shims</artifactId>
       <version>${project.version}</version>
-      <scope>runtime</scope>
     </dependency>
     <!-- inter-project -->
     <dependency>
@@ -81,6 +80,11 @@
       <artifactId>libthrift</artifactId>
       <version>${libthrift.version}</version>
     </dependency>
+    <dependency>
+      <groupId>net.sf.supercsv</groupId>
+      <artifactId>super-csv</artifactId>
+      <version>${super-csv.version}</version>
+    </dependency>
     <!-- test intra-project -->
     <dependency>
       <groupId>org.apache.hive</groupId>

Modified: hive/branches/cbo/beeline/src/java/org/apache/hive/beeline/BeeLine.java
URL: http://svn.apache.org/viewvc/hive/branches/cbo/beeline/src/java/org/apache/hive/beeline/BeeLine.java?rev=1617652&r1=1617651&r2=1617652&view=diff
==============================================================================
--- hive/branches/cbo/beeline/src/java/org/apache/hive/beeline/BeeLine.java (original)
+++ hive/branches/cbo/beeline/src/java/org/apache/hive/beeline/BeeLine.java Wed Aug 13 02:28:54 2014
@@ -79,13 +79,13 @@ import jline.ConsoleReader;
 import jline.FileNameCompletor;
 import jline.History;
 import jline.SimpleCompletor;
-import org.apache.hadoop.io.IOUtils;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.GnuParser;
 import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
+import org.apache.hadoop.io.IOUtils;
 
 
 /**
@@ -148,6 +148,7 @@ public class BeeLine implements Closeabl
       "table", new TableOutputFormat(this),
       "csv", new SeparatedValuesOutputFormat(this, ','),
       "tsv", new SeparatedValuesOutputFormat(this, '\t'),
+      "dsv", new SeparatedValuesOutputFormat(this, BeeLineOpts.DEFAULT_DELIMITER_FOR_DSV),
       "xmlattr", new XMLAttributeOutputFormat(this),
       "xmlelements", new XMLElementOutputFormat(this),
   });

Modified: hive/branches/cbo/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java
URL: http://svn.apache.org/viewvc/hive/branches/cbo/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java?rev=1617652&r1=1617651&r2=1617652&view=diff
==============================================================================
--- hive/branches/cbo/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java (original)
+++ hive/branches/cbo/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java Wed Aug 13 02:28:54 2014
@@ -53,6 +53,7 @@ class BeeLineOpts implements Completor {
   public static final String PROPERTY_NAME_EXIT =
       PROPERTY_PREFIX + "system.exit";
   public static final String DEFAULT_NULL_STRING = "NULL";
+  public static final char DEFAULT_DELIMITER_FOR_DSV = '|';
 
   private final BeeLine beeLine;
   private boolean autosave = false;
@@ -90,7 +91,7 @@ class BeeLineOpts implements Completor {
   private String scriptFile = null;
   private String initFile = null;
   private String authType = null;
-
+  private char delimiterForDSV = DEFAULT_DELIMITER_FOR_DSV;
 
   private Map<String, String> hiveVariables = new HashMap<String, String>();
   private Map<String, String> hiveConfVariables = new HashMap<String, String>();
@@ -500,5 +501,13 @@ class BeeLineOpts implements Completor {
   public void setTruncateTable(boolean truncateTable) {
     this.truncateTable = truncateTable;
   }
+
+  public char getDelimiterForDSV() {
+    return delimiterForDSV;
+  }
+
+  public void setDelimiterForDSV(char delimiterForDSV) {
+    this.delimiterForDSV = delimiterForDSV;
+  }
 }
 

Modified: hive/branches/cbo/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java
URL: http://svn.apache.org/viewvc/hive/branches/cbo/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java?rev=1617652&r1=1617651&r2=1617652&view=diff
==============================================================================
--- hive/branches/cbo/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java (original)
+++ hive/branches/cbo/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java Wed Aug 13 02:28:54 2014
@@ -48,6 +48,7 @@ import org.apache.hadoop.hive.conf.HiveC
 import org.apache.hadoop.hive.metastore.HiveMetaException;
 import org.apache.hadoop.hive.metastore.MetaStoreSchemaInfo;
 import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hive.beeline.HiveSchemaHelper.NestedScriptParser;
 
 public class HiveSchemaTool {
@@ -72,7 +73,12 @@ public class HiveSchemaTool {
     this.dbType = dbType;
     this.metaStoreSchemaInfo = new MetaStoreSchemaInfo(hiveHome, hiveConf, dbType);
     userName = hiveConf.get(ConfVars.METASTORE_CONNECTION_USER_NAME.varname);
-    passWord = hiveConf.get(HiveConf.ConfVars.METASTOREPWD.varname);
+    try {
+      passWord = ShimLoader.getHadoopShims().getPassword(hiveConf,
+          HiveConf.ConfVars.METASTOREPWD.varname);
+    } catch (IOException err) {
+      throw new HiveMetaException("Error getting metastore password", err);
+    }
   }
 
   public HiveConf getHiveConf() {

Modified: hive/branches/cbo/beeline/src/java/org/apache/hive/beeline/SeparatedValuesOutputFormat.java
URL: http://svn.apache.org/viewvc/hive/branches/cbo/beeline/src/java/org/apache/hive/beeline/SeparatedValuesOutputFormat.java?rev=1617652&r1=1617651&r2=1617652&view=diff
==============================================================================
--- hive/branches/cbo/beeline/src/java/org/apache/hive/beeline/SeparatedValuesOutputFormat.java (original)
+++ hive/branches/cbo/beeline/src/java/org/apache/hive/beeline/SeparatedValuesOutputFormat.java Wed Aug 13 02:28:54 2014
@@ -22,25 +22,43 @@
  */
 package org.apache.hive.beeline;
 
+import java.io.IOException;
+import java.io.StringWriter;
+
+import org.apache.hadoop.io.IOUtils;
+import org.supercsv.io.CsvListWriter;
+import org.supercsv.prefs.CsvPreference;
+
 /**
  * OutputFormat for values separated by a delimiter.
- *
- * <strong>TODO</strong>: Handle character escaping
- *
  */
 class SeparatedValuesOutputFormat implements OutputFormat {
   /**
    *
    */
   private final BeeLine beeLine;
-  private char separator;
+  private CsvPreference csvPreference;
 
-  public SeparatedValuesOutputFormat(BeeLine beeLine, char separator) {
+  SeparatedValuesOutputFormat(BeeLine beeLine, char separator) {
     this.beeLine = beeLine;
-    setSeparator(separator);
+    csvPreference = new CsvPreference.Builder('"', separator, "").build();
+  }
+
+  private void updateCsvPreference() {
+    if (beeLine.getOpts().getOutputFormat().equals("dsv")) {
+      // check whether delimiter changed by user
+      char curDel = (char) csvPreference.getDelimiterChar();
+      char newDel = beeLine.getOpts().getDelimiterForDSV();
+      // if delimiter changed, rebuild the csv preference
+      if (newDel != curDel) {
+        csvPreference = new CsvPreference.Builder('"', newDel, "").build();
+      }
+    }
   }
 
   public int print(Rows rows) {
+    updateCsvPreference();
+
     int count = 0;
     while (rows.hasNext()) {
       printRow(rows, (Rows.Row) rows.next());
@@ -49,23 +67,24 @@ class SeparatedValuesOutputFormat implem
     return count - 1; // sans header row
   }
 
-  public void printRow(Rows rows, Rows.Row row) {
-    String[] vals = row.values;
-    StringBuilder buf = new StringBuilder();
-    for (int i = 0; i < vals.length; i++) {
-      buf.append(buf.length() == 0 ? "" : "" + getSeparator())
-          .append('\'')
-          .append(vals[i] == null ? "" : vals[i])
-          .append('\'');
+  private String getFormattedStr(String[] vals) {
+    StringWriter strWriter = new StringWriter();
+    CsvListWriter writer = new CsvListWriter(strWriter, csvPreference);
+    if (vals.length > 0) {
+      try {
+        writer.write(vals);
+      } catch (IOException e) {
+        beeLine.error(e);
+      } finally {
+        IOUtils.closeStream(writer);
+      }
     }
-    beeLine.output(buf.toString());
-  }
-
-  public void setSeparator(char separator) {
-    this.separator = separator;
+    return strWriter.toString();
   }
 
-  public char getSeparator() {
-    return this.separator;
+  public void printRow(Rows rows, Rows.Row row) {
+    String[] vals = row.values;
+    String formattedStr = getFormattedStr(vals);
+    beeLine.output(formattedStr);
   }
 }

Modified: hive/branches/cbo/beeline/src/main/resources/BeeLine.properties
URL: http://svn.apache.org/viewvc/hive/branches/cbo/beeline/src/main/resources/BeeLine.properties?rev=1617652&r1=1617651&r2=1617652&view=diff
==============================================================================
--- hive/branches/cbo/beeline/src/main/resources/BeeLine.properties (original)
+++ hive/branches/cbo/beeline/src/main/resources/BeeLine.properties Wed Aug 13 02:28:54 2014
@@ -68,7 +68,8 @@ help-procedures: List all the procedures
 help-tables: List all the tables in the database
 help-columns: List all the columns for the specified table
 help-properties: Connect to the database specified in the properties file(s)
-help-outputformat: Set the output format for displaying results (table,vertical,csv,tsv,xmlattrs,xmlelements)
+help-outputformat: Set the output format for displaying results (table,vertical,csv,dsv,tsv,xmlattrs,xmlelements)
+help-delimiterForDSV: Set the delimiter for dsv output format
 help-nullemptystring: Set to true to get historic behavior of printing null as empty string. Default is false.
 
 jline-missing: The JLine jar was not found. Please ensure it is installed.
@@ -166,8 +167,9 @@ cmd-usage: Usage: java org.apache.hive.c
 \  --maxColumnWidth=MAXCOLWIDTH    the maximum width to use when displaying columns\n \
 \  --silent=[true/false]           be more silent\n \
 \  --autosave=[true/false]         automatically save preferences\n \
-\  --outputformat=[table/vertical/csv/tsv]   format mode for result display\n \
+\  --outputformat=[table/vertical/csv/tsv/dsv]   format mode for result display\n \
 \  --truncateTable=[true/false]    truncate table column when it exceeds length\n \
+\  --delimiterForDSV=DELIMITER     specify the delimiter for delimiter-separated values output format (default: |)\n \
 \  --isolation=LEVEL               set the transaction isolation level\n \
 \  --nullemptystring=[true/false]  set to true to get historic behavior of printing null as empty string\n \
 \  --help                          display this message

Modified: hive/branches/cbo/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java
URL: http://svn.apache.org/viewvc/hive/branches/cbo/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java?rev=1617652&r1=1617651&r2=1617652&view=diff
==============================================================================
--- hive/branches/cbo/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java (original)
+++ hive/branches/cbo/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java Wed Aug 13 02:28:54 2014
@@ -32,7 +32,6 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 
 import jline.ArgumentCompletor;
 import jline.ArgumentCompletor.AbstractArgumentDelimiter;
@@ -150,24 +149,6 @@ public class CliDriver {
             stringifyException(e));
         ret = 1;
       }
-    } else if (tokens[0].toLowerCase().equals("list")) {
-
-      SessionState.ResourceType t;
-      if (tokens.length < 2 || (t = SessionState.find_resource_type(tokens[1])) == null) {
-        console.printError("Usage: list ["
-            + StringUtils.join(SessionState.ResourceType.values(), "|") + "] [<value> [<value>]*]");
-        ret = 1;
-      } else {
-        List<String> filter = null;
-        if (tokens.length >= 3) {
-          System.arraycopy(tokens, 2, tokens, 0, tokens.length - 2);
-          filter = Arrays.asList(tokens);
-        }
-        Set<String> s = ss.list_resource(t, filter);
-        if (s != null && !s.isEmpty()) {
-          ss.out.println(StringUtils.join(s, "\n"));
-        }
-      }
     } else if (ss.isRemoteMode()) { // remote mode -- connecting to remote hive server
       HiveClient client = ss.getClient();
       PrintStream out = ss.out;

Modified: hive/branches/cbo/common/src/java/org/apache/hadoop/hive/common/FileUtils.java
URL: http://svn.apache.org/viewvc/hive/branches/cbo/common/src/java/org/apache/hadoop/hive/common/FileUtils.java?rev=1617652&r1=1617651&r2=1617652&view=diff
==============================================================================
--- hive/branches/cbo/common/src/java/org/apache/hadoop/hive/common/FileUtils.java (original)
+++ hive/branches/cbo/common/src/java/org/apache/hadoop/hive/common/FileUtils.java Wed Aug 13 02:28:54 2014
@@ -613,4 +613,19 @@ public final class FileUtils {
       return false;
     }
   }
+
+  /**
+   * @param fs1
+   * @param fs2
+   * @return return true if both file system arguments point to same file system
+   */
+  public static boolean equalsFileSystem(FileSystem fs1, FileSystem fs2) {
+    //When file system cache is disabled, you get different FileSystem objects
+    // for same file system, so '==' can't be used in such cases
+    //FileSystem api doesn't have a .equals() function implemented, so using
+    //the uri for comparison. FileSystem already uses uri+Configuration for
+    //equality in its CACHE .
+    //Once equality has been added in HDFS-4321, we should make use of it
+    return fs1.getUri().equals(fs2.getUri());
+  }
 }

Modified: hive/branches/cbo/common/src/java/org/apache/hadoop/hive/common/JavaUtils.java
URL: http://svn.apache.org/viewvc/hive/branches/cbo/common/src/java/org/apache/hadoop/hive/common/JavaUtils.java?rev=1617652&r1=1617651&r2=1617652&view=diff
==============================================================================
--- hive/branches/cbo/common/src/java/org/apache/hadoop/hive/common/JavaUtils.java (original)
+++ hive/branches/cbo/common/src/java/org/apache/hadoop/hive/common/JavaUtils.java Wed Aug 13 02:28:54 2014
@@ -124,6 +124,7 @@ public final class JavaUtils {
         newOutputStream.close();
       }
     }
+    LogFactory.release(loader);
   }
 
   private JavaUtils() {

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=1617652&r1=1617651&r2=1617652&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 Wed Aug 13 02:28:54 2014
@@ -900,7 +900,12 @@ public class HiveConf extends Configurat
         "This controls how many partitions can be scanned for each partitioned table.\n" +
         "The default value \"-1\" means no limit."),
 
-    HIVEHASHTABLETHRESHOLD("hive.hashtable.initialCapacity", 100000, ""),
+    HIVEHASHTABLEKEYCOUNTADJUSTMENT("hive.hashtable.key.count.adjustment", 1.0f,
+        "Adjustment to mapjoin hashtable size derived from table and column statistics; the estimate" +
+        " of the number of keys is divided by this value. If the value is 0, statistics are not used" +
+        "and hive.hashtable.initialCapacity is used instead."),
+    HIVEHASHTABLETHRESHOLD("hive.hashtable.initialCapacity", 100000, "Initial capacity of " +
+        "mapjoin hashtable if statistics are absent, or if hive.hashtable.stats.key.estimate.adjustment is set to 0"),
     HIVEHASHTABLELOADFACTOR("hive.hashtable.loadfactor", (float) 0.75, ""),
     HIVEHASHTABLEFOLLOWBYGBYMAXMEMORYUSAGE("hive.mapjoin.followby.gby.localtask.max.memory.usage", (float) 0.55,
         "This number means how much memory the local task can take to hold the key/value into an in-memory hash table \n" +
@@ -1554,7 +1559,7 @@ public class HiveConf extends Configurat
     HIVE_SERVER2_SSL_KEYSTORE_PATH("hive.server2.keystore.path", "", ""),
     HIVE_SERVER2_SSL_KEYSTORE_PASSWORD("hive.server2.keystore.password", "", ""),
 
-    HIVE_SECURITY_COMMAND_WHITELIST("hive.security.command.whitelist", "set,reset,dfs,add,delete,compile",
+    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_CONF_RESTRICTED_LIST("hive.conf.restricted.list",

Modified: hive/branches/cbo/conf/hive-default.xml.template
URL: http://svn.apache.org/viewvc/hive/branches/cbo/conf/hive-default.xml.template?rev=1617652&r1=1617651&r2=1617652&view=diff
==============================================================================
--- hive/branches/cbo/conf/hive-default.xml.template (original)
+++ hive/branches/cbo/conf/hive-default.xml.template Wed Aug 13 02:28:54 2014
@@ -933,21 +933,6 @@
     <description>Flag to control enabling Cost Based Optimizations using Optiq framework.</description>
   </property>
   <property>
-    <key>hive.cbo.max.joins.supported</key>
-    <value>10</value>
-    <description> Control queries that will be considered for join reordering, based on number of joins in them. Beyond a certain number of joins, the cost of considering possible permutations is prohibitive.</description>
-  </property>
-  <property>
-    <key>hive.cbo.project.pullabovejoin.rule</key>
-    <value>false</value>
-    <description/>
-  </property>
-  <property>
-    <key>hive.cbo.greedy.join.order</key>
-    <value>false</value>
-    <description/>
-  </property>
-  <property>
     <key>hive.mapjoin.bucket.cache.size</key>
     <value>100</value>
     <description/>
@@ -1551,9 +1536,14 @@
     </description>
   </property>
   <property>
+    <key>hive.hashtable.key.count.adjustment</key>
+    <value>1.0</value>
+    <description>Adjustment to mapjoin hashtable size derived from table and column statistics; the estimate of the number of keys is divided by this value. If the value is 0, statistics are not usedand hive.hashtable.initialCapacity is used instead.</description>
+  </property>
+  <property>
     <key>hive.hashtable.initialCapacity</key>
     <value>100000</value>
-    <description/>
+    <description>Initial capacity of mapjoin hashtable if statistics are absent, or if hive.hashtable.stats.key.estimate.adjustment is set to 0</description>
   </property>
   <property>
     <key>hive.hashtable.loadfactor</key>
@@ -2819,7 +2809,7 @@
   </property>
   <property>
     <key>hive.security.command.whitelist</key>
-    <value>set,reset,dfs,add,delete,compile</value>
+    <value>set,reset,dfs,add,list,delete,compile</value>
     <description>Comma separated list of non-SQL Hive commands users are authorized to execute</description>
   </property>
   <property>

Modified: hive/branches/cbo/data/conf/hive-site.xml
URL: http://svn.apache.org/viewvc/hive/branches/cbo/data/conf/hive-site.xml?rev=1617652&r1=1617651&r2=1617652&view=diff
==============================================================================
--- hive/branches/cbo/data/conf/hive-site.xml (original)
+++ hive/branches/cbo/data/conf/hive-site.xml Wed Aug 13 02:28:54 2014
@@ -112,6 +112,12 @@
 </property>
 
 <property>
+  <name>test.data.scripts</name>
+  <value>${hive.root}/data/scripts</value>
+  <description></description>
+</property>
+
+<property>
   <name>hive.jar.path</name>
   <value>${maven.local.repository}/org/apache/hive/hive-exec/${hive.version}/hive-exec-${hive.version}.jar</value>
   <description></description>

Modified: hive/branches/cbo/data/conf/tez/hive-site.xml
URL: http://svn.apache.org/viewvc/hive/branches/cbo/data/conf/tez/hive-site.xml?rev=1617652&r1=1617651&r2=1617652&view=diff
==============================================================================
Binary files - no diff available.

Modified: hive/branches/cbo/data/scripts/q_test_cleanup.sql
URL: http://svn.apache.org/viewvc/hive/branches/cbo/data/scripts/q_test_cleanup.sql?rev=1617652&r1=1617651&r2=1617652&view=diff
==============================================================================
--- hive/branches/cbo/data/scripts/q_test_cleanup.sql (original)
+++ hive/branches/cbo/data/scripts/q_test_cleanup.sql Wed Aug 13 02:28:54 2014
@@ -7,4 +7,12 @@ DROP TABLE IF EXISTS srcbucket;
 DROP TABLE IF EXISTS srcbucket2;
 DROP TABLE IF EXISTS srcpart;
 DROP TABLE IF EXISTS primitives;
-
+DROP TABLE IF EXISTS dest1;
+DROP TABLE IF EXISTS dest2;
+DROP TABLE IF EXISTS dest3;
+DROP TABLE IF EXISTS dest4;
+DROP TABLE IF EXISTS dest4_sequencefile;
+DROP TABLE IF EXISTS dest_j1;
+DROP TABLE IF EXISTS dest_g1;
+DROP TABLE IF EXISTS dest_g2;
+DROP TABLE IF EXISTS fetchtask_ioexception;

Modified: hive/branches/cbo/data/scripts/q_test_init.sql
URL: http://svn.apache.org/viewvc/hive/branches/cbo/data/scripts/q_test_init.sql?rev=1617652&r1=1617651&r2=1617652&view=diff
==============================================================================
--- hive/branches/cbo/data/scripts/q_test_init.sql (original)
+++ hive/branches/cbo/data/scripts/q_test_init.sql Wed Aug 13 02:28:54 2014
@@ -3,7 +3,7 @@
 --
 DROP TABLE IF EXISTS src;
 
-CREATE TABLE src (key STRING, value STRING) STORED AS TEXTFILE;
+CREATE TABLE src (key STRING COMMENT 'default', value STRING COMMENT 'default') STORED AS TEXTFILE;
 
 LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/kv1.txt" INTO TABLE src;
 
@@ -12,7 +12,7 @@ LOAD DATA LOCAL INPATH "${hiveconf:test.
 --
 DROP TABLE IF EXISTS src1;
 
-CREATE TABLE src1 (key STRING, value STRING) STORED AS TEXTFILE;
+CREATE TABLE src1 (key STRING COMMENT 'default', value STRING COMMENT 'default') STORED AS TEXTFILE;
 
 LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/kv3.txt" INTO TABLE src1;
 
@@ -21,7 +21,7 @@ LOAD DATA LOCAL INPATH "${hiveconf:test.
 --
 DROP TABLE IF EXISTS src_json;
 
-CREATE TABLE src_json (json STRING) STORED AS TEXTFILE;
+CREATE TABLE src_json (json STRING COMMENT 'default') STORED AS TEXTFILE;
 
 LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/json.txt" INTO TABLE src_json;
 
@@ -31,7 +31,7 @@ LOAD DATA LOCAL INPATH "${hiveconf:test.
 --
 DROP TABLE IF EXISTS src_sequencefile;
 
-CREATE TABLE src_sequencefile (key STRING, value STRING) STORED AS SEQUENCEFILE;
+CREATE TABLE src_sequencefile (key STRING COMMENT 'default', value STRING COMMENT 'default') STORED AS SEQUENCEFILE;
 
 LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/kv1.seq" INTO TABLE src_sequencefile;
 
@@ -45,7 +45,7 @@ CREATE TABLE src_thrift
 ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer'
 WITH SERDEPROPERTIES (
   'serialization.class' = 'org.apache.hadoop.hive.serde2.thrift.test.Complex',
-  'serialization.format' = 'com.facebook.thrift.protocol.TBinaryProtocol')
+  'serialization.format' = 'org.apache.thrift.protocol.TBinaryProtocol')
 STORED AS SEQUENCEFILE;
 
 LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/complex.seq" INTO TABLE src_thrift;
@@ -75,6 +75,8 @@ STORED AS TEXTFILE;
 
 LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/srcbucket20.txt" INTO TABLE srcbucket2;
 LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/srcbucket21.txt" INTO TABLE srcbucket2;
+LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/srcbucket22.txt" INTO TABLE srcbucket2;
+LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/srcbucket23.txt" INTO TABLE srcbucket2;
 
 
 --
@@ -82,7 +84,7 @@ LOAD DATA LOCAL INPATH "${hiveconf:test.
 --
 DROP TABLE IF EXISTS srcpart;
 
-CREATE TABLE srcpart (key STRING, value STRING)
+CREATE TABLE srcpart (key STRING COMMENT 'default', value STRING COMMENT 'default')
 PARTITIONED BY (ds STRING, hr STRING)
 STORED AS TEXTFILE;
 
@@ -99,20 +101,46 @@ LOAD DATA LOCAL INPATH "${hiveconf:test.
 OVERWRITE INTO TABLE srcpart PARTITION (ds="2008-04-09", hr="12");
 
 
+--
+-- Table alltypesorc
+--
+DROP TABLE IF EXISTS alltypesorc;
+CREATE TABLE alltypesorc(
+    ctinyint TINYINT,
+    csmallint SMALLINT,
+    cint INT,
+    cbigint BIGINT,
+    cfloat FLOAT,
+    cdouble DOUBLE,
+    cstring1 STRING,
+    cstring2 STRING,
+    ctimestamp1 TIMESTAMP,
+    ctimestamp2 TIMESTAMP,
+    cboolean1 BOOLEAN,
+    cboolean2 BOOLEAN)
+    STORED AS ORC;
+
+LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/alltypesorc"
+OVERWRITE INTO  TABLE alltypesorc;
+
+
+--
+-- Table primitives
+--
 DROP TABLE IF EXISTS primitives;
 CREATE TABLE primitives (
-  id INT,
-  bool_col BOOLEAN,
-  tinyint_col TINYINT,
-  smallint_col SMALLINT,
-  int_col INT,
-  bigint_col BIGINT,
-  float_col FLOAT,
-  double_col DOUBLE,
-  date_string_col STRING,
-  string_col STRING,
-  timestamp_col TIMESTAMP)
-PARTITIONED BY (year INT, month INT)
+  id INT COMMENT 'default',
+  bool_col BOOLEAN COMMENT 'default',
+  tinyint_col TINYINT COMMENT 'default',
+  smallint_col SMALLINT COMMENT 'default',
+  int_col INT COMMENT 'default',
+  bigint_col BIGINT COMMENT 'default',
+  float_col FLOAT COMMENT 'default',
+  double_col DOUBLE COMMENT 'default',
+  date_string_col STRING COMMENT 'default',
+  string_col STRING COMMENT 'default',
+  timestamp_col TIMESTAMP COMMENT 'default')
+PARTITIONED BY (year INT COMMENT 'default', month INT COMMENT 'default')
 ROW FORMAT DELIMITED
   FIELDS TERMINATED BY ','
   ESCAPED BY '\\'
@@ -130,3 +158,60 @@ OVERWRITE INTO TABLE primitives PARTITIO
 LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/types/primitives/090401.txt"
 OVERWRITE INTO TABLE primitives PARTITION(year=2009, month=4);
 
+--
+-- Function qtest_get_java_boolean
+--
+DROP FUNCTION IF EXISTS qtest_get_java_boolean;
+CREATE FUNCTION qtest_get_java_boolean AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFTestGetJavaBoolean';
+
+--
+-- Table dest1
+--
+DROP TABLE IF EXISTS dest1;
+
+CREATE TABLE dest1 (key STRING COMMENT 'default', value STRING COMMENT 'default')
+STORED AS
+INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat'
+OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat';
+
+--
+-- Table dest2
+--
+DROP TABLE IF EXISTS dest2;
+
+CREATE TABLE dest2 (key STRING COMMENT 'default', value STRING COMMENT 'default')
+STORED AS
+INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat'
+OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat';
+
+--
+-- Table dest3
+--
+DROP TABLE IF EXISTS dest3;
+
+CREATE TABLE dest3 (key STRING COMMENT 'default', value STRING COMMENT 'default')
+PARTITIONED BY (ds STRING, hr STRING)
+STORED AS
+INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat'
+OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat';
+ALTER TABLE dest3 ADD PARTITION (ds='2008-04-08',hr='12');
+
+--
+-- Table dest4
+--
+DROP TABLE IF EXISTS dest4;
+
+CREATE TABLE dest4 (key STRING COMMENT 'default', value STRING COMMENT 'default')
+STORED AS
+INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat'
+OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat';
+
+--
+-- Table dest4_sequencefile
+--
+DROP TABLE IF EXISTS dest4_sequencefile;
+
+CREATE TABLE dest4_sequencefile (key STRING COMMENT 'default', value STRING COMMENT 'default')
+STORED AS
+INPUTFORMAT 'org.apache.hadoop.mapred.SequenceFileInputFormat'
+OUTPUTFORMAT 'org.apache.hadoop.mapred.SequenceFileOutputFormat';
\ No newline at end of file

Modified: hive/branches/cbo/hbase-handler/src/test/templates/TestHBaseCliDriver.vm
URL: http://svn.apache.org/viewvc/hive/branches/cbo/hbase-handler/src/test/templates/TestHBaseCliDriver.vm?rev=1617652&r1=1617651&r2=1617652&view=diff
==============================================================================
--- hive/branches/cbo/hbase-handler/src/test/templates/TestHBaseCliDriver.vm (original)
+++ hive/branches/cbo/hbase-handler/src/test/templates/TestHBaseCliDriver.vm Wed Aug 13 02:28:54 2014
@@ -44,9 +44,12 @@ public class $className extends TestCase
   protected void setUp() {
 
     MiniClusterType miniMR = MiniClusterType.valueForString("$clusterMode");
+    String initScript = "$initScript";
+    String cleanupScript = "$cleanupScript";
 
     try {
-      qt = new HBaseQTestUtil((HIVE_ROOT + "$resultsDir"), (HIVE_ROOT + "$logDir"), miniMR, setup);
+      qt = new HBaseQTestUtil((HIVE_ROOT + "$resultsDir"), (HIVE_ROOT + "$logDir"), miniMR,
+      setup, initScript, cleanupScript);
     } catch (Exception e) {
       System.err.println("Exception: " + e.getMessage());
       e.printStackTrace();

Modified: hive/branches/cbo/hbase-handler/src/test/templates/TestHBaseNegativeCliDriver.vm
URL: http://svn.apache.org/viewvc/hive/branches/cbo/hbase-handler/src/test/templates/TestHBaseNegativeCliDriver.vm?rev=1617652&r1=1617651&r2=1617652&view=diff
==============================================================================
--- hive/branches/cbo/hbase-handler/src/test/templates/TestHBaseNegativeCliDriver.vm (original)
+++ hive/branches/cbo/hbase-handler/src/test/templates/TestHBaseNegativeCliDriver.vm Wed Aug 13 02:28:54 2014
@@ -45,9 +45,12 @@ public class $className extends TestCase
   protected void setUp() {
 
     MiniClusterType miniMR = MiniClusterType.valueForString("$clusterMode");
+    String initScript = "$initScript";
+    String cleanupScript = "$cleanupScript";
 
     try {
-      qt = new HBaseQTestUtil((HIVE_ROOT + "$resultsDir"), (HIVE_ROOT + "$logDir"), miniMR, setup);
+      qt = new HBaseQTestUtil((HIVE_ROOT + "$resultsDir"), (HIVE_ROOT + "$logDir"), miniMR,
+      setup, initScript, cleanupScript);
     } catch (Exception e) {
       System.err.println("Exception: " + e.getMessage());
       e.printStackTrace();

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=1617652&r1=1617651&r2=1617652&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 Wed Aug 13 02:28:54 2014
@@ -20,6 +20,7 @@ package org.apache.hive.hcatalog.mapredu
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.io.RCFileOutputFormat;
 import org.apache.hadoop.hive.ql.io.orc.OrcFile;
@@ -37,6 +38,7 @@ import java.util.Map;
  * class that allows us to still be as generic as possible
  * in the main codeflow path, and call attention to the special
  * cases here.
+ *
  * Note : For all methods introduced here, please document why
  * the special case is necessary, providing a jira number if
  * possible.
@@ -50,6 +52,11 @@ public class SpecialCases {
    * instantiating a storage handler to write. We set any parameters
    * we want to be visible to the job in jobProperties, and this will
    * be available to the job via jobconf at run time.
+   *
+   * This is mostly intended to be used by StorageHandlers that wrap
+   * File-based OutputFormats such as FosterStorageHandler that wraps
+   * RCFile, ORC, etc.
+   *
    * @param jobProperties : map to write to
    * @param jobInfo : information about this output job to read from
    * @param ofclass : the output format in use
@@ -78,5 +85,26 @@ public class SpecialCases {
     }
   }
 
+  /**
+   * Method to do any storage-handler specific special casing while instantiating a
+   * HCatLoader
+   *
+   * @param conf : configuration to write to
+   * @param tableInfo : the table definition being used
+   */
+  public static void addSpecialCasesParametersForHCatLoader(
+      Configuration conf, HCatTableInfo tableInfo) {
+    if ((tableInfo == null) || (tableInfo.getStorerInfo() == null)){
+      return;
+    }
+    String shClass = tableInfo.getStorerInfo().getStorageHandlerClass();
+    if ((shClass != null) && shClass.equals("org.apache.hadoop.hive.hbase.HBaseStorageHandler")){
+      // NOTE: The reason we use a string name of the hive hbase handler here is
+      // because we do not want to introduce a compile-dependency on the hive-hbase-handler
+      // module from within hive-hcatalog.
+      // This parameter was added due to the requirement in HIVE-7072
+      conf.set("pig.noSplitCombination", "true");
+    }
+  }
 
 }

Modified: hive/branches/cbo/hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hive/hcatalog/pig/HCatLoader.java
URL: http://svn.apache.org/viewvc/hive/branches/cbo/hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hive/hcatalog/pig/HCatLoader.java?rev=1617652&r1=1617651&r2=1617652&view=diff
==============================================================================
--- hive/branches/cbo/hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hive/hcatalog/pig/HCatLoader.java (original)
+++ hive/branches/cbo/hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hive/hcatalog/pig/HCatLoader.java Wed Aug 13 02:28:54 2014
@@ -43,6 +43,7 @@ import org.apache.hive.hcatalog.data.Pai
 import org.apache.hive.hcatalog.data.schema.HCatSchema;
 import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
 import org.apache.hive.hcatalog.mapreduce.InputJobInfo;
+import org.apache.hive.hcatalog.mapreduce.SpecialCases;
 import org.apache.pig.Expression;
 import org.apache.pig.Expression.BinaryExpression;
 import org.apache.pig.PigException;
@@ -125,6 +126,12 @@ public class HCatLoader extends HCatBase
       Job clone = new Job(job.getConfiguration());
       HCatInputFormat.setInput(job, dbName, tableName, getPartitionFilterString());
 
+      InputJobInfo inputJobInfo = (InputJobInfo) HCatUtil.deserialize(
+          job.getConfiguration().get(HCatConstants.HCAT_KEY_JOB_INFO));
+
+      SpecialCases.addSpecialCasesParametersForHCatLoader(job.getConfiguration(),
+          inputJobInfo.getTableInfo());
+
       // We will store all the new /changed properties in the job in the
       // udf context, so the the HCatInputFormat.setInput method need not
       //be called many times.

Modified: hive/branches/cbo/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestLocationQueries.java
URL: http://svn.apache.org/viewvc/hive/branches/cbo/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestLocationQueries.java?rev=1617652&r1=1617651&r2=1617652&view=diff
==============================================================================
--- hive/branches/cbo/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestLocationQueries.java (original)
+++ hive/branches/cbo/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestLocationQueries.java Wed Aug 13 02:28:54 2014
@@ -88,7 +88,7 @@ public class TestLocationQueries extends
         String hadoopVer, String locationSubdir)
       throws Exception
     {
-      super(outDir, logDir, miniMr, hadoopVer);
+      super(outDir, logDir, miniMr, hadoopVer, "", "");
       this.locationSubdir = locationSubdir;
     }
   }

Modified: hive/branches/cbo/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java
URL: http://svn.apache.org/viewvc/hive/branches/cbo/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java?rev=1617652&r1=1617651&r2=1617652&view=diff
==============================================================================
--- hive/branches/cbo/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java (original)
+++ hive/branches/cbo/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java Wed Aug 13 02:28:54 2014
@@ -29,7 +29,6 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
-import java.util.Set;
 
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
@@ -110,7 +109,7 @@ public class TestHiveAuthorizerCheckInvo
         getSortedList(tableObj.getColumns()));
   }
 
-  private List<String> getSortedList(Set<String> columns) {
+  private List<String> getSortedList(List<String> columns) {
     List<String> sortedCols = new ArrayList<String>(columns);
     Collections.sort(sortedCols);
     return sortedCols;

Modified: hive/branches/cbo/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java
URL: http://svn.apache.org/viewvc/hive/branches/cbo/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java?rev=1617652&r1=1617651&r2=1617652&view=diff
==============================================================================
--- hive/branches/cbo/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java (original)
+++ hive/branches/cbo/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java Wed Aug 13 02:28:54 2014
@@ -362,7 +362,7 @@ public class TestBeeLineWithArgs {
     final String TEST_NAME = "testNullNonDefault";
     final String SCRIPT_TEXT = "set hive.support.concurrency = false;\n" +
                 "!set nullemptystring true\n select 'abc',null,'def' from " + tableName + " limit 1 ;\n";
-    final String EXPECTED_PATTERN = "'abc','','def'";
+    final String EXPECTED_PATTERN = "abc,,def";
 
     List<String> argList = getBaseArgs(JDBC_URL);
     argList.add("--outputformat=csv");
@@ -382,7 +382,7 @@ public class TestBeeLineWithArgs {
     final String SCRIPT_TEXT = "set hive.support.concurrency = false;\n" +
                 "select 'abc',null,'def' from " + tableName + " limit 1 ;\n";
     //final String EXPECTED_PATTERN = "| abc  |      | def  |";
-    final String EXPECTED_PATTERN = "'abc','','def'";
+    final String EXPECTED_PATTERN = "abc,,def";
 
     List<String> argList = getBaseArgs(JDBC_URL);
     argList.add("--nullemptystring=true");

Modified: hive/branches/cbo/itests/qtest/pom.xml
URL: http://svn.apache.org/viewvc/hive/branches/cbo/itests/qtest/pom.xml?rev=1617652&r1=1617651&r2=1617652&view=diff
==============================================================================
--- hive/branches/cbo/itests/qtest/pom.xml (original)
+++ hive/branches/cbo/itests/qtest/pom.xml Wed Aug 13 02:28:54 2014
@@ -373,7 +373,7 @@
             </goals>
             <configuration>
               <files>
-                <file>${basedir}/testconfiguration.properties</file>
+                <file>${basedir}/../src/test/resources/testconfiguration.properties</file>
               </files>
             </configuration>
           </execution>
@@ -420,7 +420,9 @@
                   resultsDirectory="${basedir}/${hive.path.to.root}/ql/src/test/results/compiler/" className="TestParse"
                   logFile="${project.build.directory}/testparsegen.log"
                   hadoopVersion="${active.hadoop.version}"
-                  logDirectory="${project.build.directory}/qfile-results/positive/"/>
+                  logDirectory="${project.build.directory}/qfile-results/positive/"
+                  initScript="q_test_init.sql"
+                  cleanupScript="q_test_cleanup.sql"/>
 
                 <!-- Negative Parse -->
                 <qtestgen hiveRootDirectory="${basedir}/${hive.path.to.root}/"
@@ -433,7 +435,9 @@
                   resultsDirectory="${basedir}/${hive.path.to.root}/ql/src/test/results/compiler/errors/" className="TestParseNegative"
                   logFile="${project.build.directory}/testparseneggen.log"
                   hadoopVersion="${active.hadoop.version}"
-                  logDirectory="${project.build.directory}/qfile-results/negative/"/>
+                  logDirectory="${project.build.directory}/qfile-results/negative/"
+                  initScript="q_test_init.sql"
+                  cleanupScript="q_test_cleanup.sql"/>
 
                 <!-- Cli -->
                 <qtestgen hiveRootDirectory="${basedir}/${hive.path.to.root}/"
@@ -448,7 +452,9 @@
                   resultsDirectory="${basedir}/${hive.path.to.root}/ql/src/test/results/clientpositive/" className="TestCliDriver"
                   logFile="${project.build.directory}/testclidrivergen.log"
                   logDirectory="${project.build.directory}/qfile-results/clientpositive/"
-                  hadoopVersion="${active.hadoop.version}"/>
+                  hadoopVersion="${active.hadoop.version}"
+                  initScript="q_test_init.sql"
+                  cleanupScript="q_test_cleanup.sql"/>
 
                 <!-- Negative Cli -->
                 <qtestgen hiveRootDirectory="${basedir}/${hive.path.to.root}/"
@@ -463,7 +469,9 @@
                   resultsDirectory="${basedir}/${hive.path.to.root}/ql/src/test/results/clientnegative/" className="TestNegativeCliDriver"
                   logFile="${project.build.directory}/testnegativeclidrivergen.log"
                   logDirectory="${project.build.directory}/qfile-results/clientnegative/"
-                  hadoopVersion="${active.hadoop.version}"/>
+                  hadoopVersion="${active.hadoop.version}"
+                  initScript="q_test_init.sql"
+                  cleanupScript="q_test_cleanup.sql"/>
 
                 <!-- Compare Cli -->
                 <qtestgen hiveRootDirectory="${basedir}/${hive.path.to.root}/"
@@ -477,7 +485,9 @@
                   className="TestCompareCliDriver"
                   logFile="${project.build.directory}/testcompareclidrivergen.log"
                   logDirectory="${project.build.directory}/qfile-results/clientcompare/"
-                  hadoopVersion="${active.hadoop.version}"/>
+                  hadoopVersion="${active.hadoop.version}"
+                  initScript="q_test_init.sql"
+                  cleanupScript="q_test_cleanup.sql"/>
 
                 <!-- Minimr -->
                 <qtestgen hiveRootDirectory="${basedir}/${hive.path.to.root}/"
@@ -493,7 +503,8 @@
                   logFile="${project.build.directory}/testminimrclidrivergen.log"
                   logDirectory="${project.build.directory}/qfile-results/clientpositive/"
                   hadoopVersion="${active.hadoop.version}"
-                  />
+                  initScript="q_test_init.sql"
+                  cleanupScript="q_test_cleanup.sql"/>
 
                 <if>
                   <equals arg1="${active.hadoop.version}" arg2="${hadoop-23.version}"/>
@@ -513,7 +524,8 @@
                               logFile="${project.build.directory}/testminitezclidrivergen.log"
                               logDirectory="${project.build.directory}/qfile-results/clientpositive/"
                               hadoopVersion="${active.hadoop.version}"
-                              />
+                              initScript="q_test_init.sql"
+                              cleanupScript="q_test_cleanup.sql"/>
                   </then>
                   <else>
                   </else>
@@ -533,7 +545,8 @@
                   logFile="${project.build.directory}/testnegativeminimrclidrivergen.log"
                   logDirectory="${project.build.directory}/qfile-results/clientnegative/"
                   hadoopVersion="${hadoopVersion}"
-                  />
+                  initScript="q_test_init.sql"
+                  cleanupScript="q_test_cleanup.sql"/>
 
                 <!-- HBase Positive -->
                 <qtestgen hiveRootDirectory="${basedir}/${hive.path.to.root}/"
@@ -545,7 +558,9 @@
                   clusterMode="${clustermode}"
                   resultsDirectory="${basedir}/${hive.path.to.root}/hbase-handler/src/test/results/positive/" className="TestHBaseCliDriver"
                   logFile="${project.build.directory}/testhbaseclidrivergen.log"
-                  logDirectory="${project.build.directory}/qfile-results/hbase-handler/positive/"/>
+                  logDirectory="${project.build.directory}/qfile-results/hbase-handler/positive/"
+                  initScript="q_test_init.sql"
+                  cleanupScript="q_test_cleanup.sql"/>
 
                 <!-- HBase Minimr -->
                 <qtestgen hiveRootDirectory="${basedir}/${hive.path.to.root}/"
@@ -557,7 +572,9 @@
                   clusterMode="miniMR"
                   resultsDirectory="${basedir}/${hive.path.to.root}/hbase-handler/src/test/results/positive/" className="TestHBaseMinimrCliDriver"
                   logFile="${project.build.directory}/testhbaseminimrclidrivergen.log"
-                  logDirectory="${project.build.directory}/qfile-results/hbase-handler/minimrpositive/"/>
+                  logDirectory="${project.build.directory}/qfile-results/hbase-handler/minimrpositive/"
+                  initScript="q_test_init.sql"
+                  cleanupScript="q_test_cleanup.sql"/>
 
                 <!-- HBase Negative -->
                 <qtestgen hiveRootDirectory="${basedir}/${hive.path.to.root}/"
@@ -569,7 +586,9 @@
                   clusterMode="${clustermode}"
                   resultsDirectory="${basedir}/${hive.path.to.root}/hbase-handler/src/test/results/negative/" className="TestHBaseNegativeCliDriver"
                   logFile="${project.build.directory}/testhbasenegativeclidrivergen.log"
-                  logDirectory="${project.build.directory}/qfile-results/hbase-handler/negative"/>
+                  logDirectory="${project.build.directory}/qfile-results/hbase-handler/negative"
+                  initScript="q_test_init.sql"
+                  cleanupScript="q_test_cleanup.sql"/>
 
 
                 <!-- Beeline -->
@@ -588,7 +607,7 @@
                         resultsDirectory="${basedir}/${hive.path.to.root}/ql/src/test/results/clientpositive/" className="TestBeeLineDriver"
                         logFile="${project.build.directory}/testbeelinedrivergen.log"
                         logDirectory="${project.build.directory}/qfile-results/beelinepositive/"
-                        hadoopVersion="${hadoopVersion}" />
+                        hadoopVersion="${hadoopVersion}"/>
                     </then>
                 </if>
 
@@ -606,7 +625,8 @@
                   logFile="${project.build.directory}/testcontribclidrivergen.log"
                   logDirectory="${project.build.directory}/qfile-results/contribclientpositive"
                   hadoopVersion="${hadoopVersion}"
-                />
+                  initScript="q_test_init.sql"
+                  cleanupScript="q_test_cleanup.sql"/>
 
                 <qtestgen hiveRootDirectory="${basedir}/${hive.path.to.root}/"
                   outputDirectory="${project.build.directory}/generated-test-sources/java/org/apache/hadoop/hive/cli"
@@ -617,7 +637,9 @@
                   runDisabled="${run_disabled}"
                   resultsDirectory="${basedir}/${hive.path.to.root}/contrib/src/test/results/clientnegative/" className="TestContribNegativeCliDriver"
                   logFile="${project.build.directory}/testcontribnegclidrivergen.log"
-                  logDirectory="${project.build.directory}/qfile-results/contribclientnegative"/>
+                  logDirectory="${project.build.directory}/qfile-results/contribclientnegative"
+                  initScript="q_test_init.sql"
+                  cleanupScript="q_test_cleanup.sql"/>
 
 
               </target>

Modified: hive/branches/cbo/itests/src/test/resources/testconfiguration.properties
URL: http://svn.apache.org/viewvc/hive/branches/cbo/itests/src/test/resources/testconfiguration.properties?rev=1617652&r1=1617650&r2=1617652&view=diff
==============================================================================
--- hive/branches/cbo/itests/src/test/resources/testconfiguration.properties (original)
+++ hive/branches/cbo/itests/src/test/resources/testconfiguration.properties Wed Aug 13 02:28:54 2014
@@ -54,6 +54,7 @@ minitez.query.files.shared=alter_merge_2
   bucket2.q,\
   bucket3.q,\
   bucket4.q,\
+  cbo_correctness.q,\
   count.q,\
   create_merge_compressed.q,\
   cross_join.q,\

Modified: hive/branches/cbo/itests/util/src/main/java/org/apache/hadoop/hive/hbase/HBaseQTestUtil.java
URL: http://svn.apache.org/viewvc/hive/branches/cbo/itests/util/src/main/java/org/apache/hadoop/hive/hbase/HBaseQTestUtil.java?rev=1617652&r1=1617651&r2=1617652&view=diff
==============================================================================
--- hive/branches/cbo/itests/util/src/main/java/org/apache/hadoop/hive/hbase/HBaseQTestUtil.java (original)
+++ hive/branches/cbo/itests/util/src/main/java/org/apache/hadoop/hive/hbase/HBaseQTestUtil.java Wed Aug 13 02:28:54 2014
@@ -40,10 +40,11 @@ public class HBaseQTestUtil extends QTes
   private final HConnection conn;
 
   public HBaseQTestUtil(
-    String outDir, String logDir, MiniClusterType miniMr, HBaseTestSetup setup)
+    String outDir, String logDir, MiniClusterType miniMr, HBaseTestSetup setup,
+    String initScript, String cleanupScript)
     throws Exception {
 
-    super(outDir, logDir, miniMr, null);
+    super(outDir, logDir, miniMr, null, initScript, cleanupScript);
     setup.preTest(conf);
     this.conn = setup.getConnection();
     super.init();

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=1617652&r1=1617651&r2=1617652&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 Wed Aug 13 02:28:54 2014
@@ -38,7 +38,6 @@ import java.io.OutputStreamWriter;
 import java.io.PrintStream;
 import java.io.Serializable;
 import java.io.StringWriter;
-import java.io.UnsupportedEncodingException;
 import java.net.URL;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -115,6 +114,8 @@ public class QTestUtil {
 
   public static final String UTF_8 = "UTF-8";
   private static final Log LOG = LogFactory.getLog("QTestUtil");
+  private final String defaultInitScript = "q_test_init.sql";
+  private final String defaultCleanupScript = "q_test_cleanup.sql";
 
   private String testWarehouse;
   private final String testFiles;
@@ -142,6 +143,10 @@ public class QTestUtil {
   private boolean miniMr = false;
   private String hadoopVer = null;
   private QTestSetup setup = null;
+  private boolean isSessionStateStarted = false;
+
+  private String initScript;
+  private String cleanupScript;
 
   static {
     for (String srcTable : System.getProperty("test.src.tables", "").trim().split(",")) {
@@ -225,8 +230,9 @@ public class QTestUtil {
     }
   }
 
-  public QTestUtil(String outDir, String logDir) throws Exception {
-    this(outDir, logDir, MiniClusterType.none, null, "0.20");
+  public QTestUtil(String outDir, String logDir, String initScript, String cleanupScript) throws
+      Exception {
+    this(outDir, logDir, MiniClusterType.none, null, "0.20", initScript, cleanupScript);
   }
 
   public String getOutputDirectory() {
@@ -297,13 +303,14 @@ public class QTestUtil {
     }
   }
 
-  public QTestUtil(String outDir, String logDir, MiniClusterType clusterType, String hadoopVer)
+  public QTestUtil(String outDir, String logDir, MiniClusterType clusterType, String hadoopVer,
+                   String initScript, String cleanupScript)
     throws Exception {
-    this(outDir, logDir, clusterType, null, hadoopVer);
+    this(outDir, logDir, clusterType, null, hadoopVer, initScript, cleanupScript);
   }
 
   public QTestUtil(String outDir, String logDir, MiniClusterType clusterType,
-      String confDir, String hadoopVer)
+      String confDir, String hadoopVer, String initScript, String cleanupScript)
     throws Exception {
     this.outDir = outDir;
     this.logDir = logDir;
@@ -354,6 +361,20 @@ public class QTestUtil {
 
     testFiles = dataDir;
 
+    // Use the current directory if it is not specified
+    String scriptsDir = conf.get("test.data.scripts");
+    if (scriptsDir == null) {
+      scriptsDir = new File(".").getAbsolutePath() + "/data/scripts";
+    }
+    if (initScript.isEmpty()) {
+      initScript = defaultInitScript;
+    }
+    if (cleanupScript.isEmpty()) {
+      cleanupScript = defaultCleanupScript;
+    }
+    this.initScript = scriptsDir + "/" + initScript;
+    this.cleanupScript = scriptsDir + "/" + cleanupScript;
+
     overWrite = "true".equalsIgnoreCase(System.getProperty("test.output.overwrite"));
 
     setup = new QTestSetup();
@@ -593,14 +614,15 @@ public class QTestUtil {
   }
 
   public void cleanUp() throws Exception {
-    // Drop any tables that remain due to unsuccessful runs
-    for (String s : new String[] {"src", "src1", "src_json", "src_thrift",
-        "src_sequencefile", "srcpart", "srcbucket", "srcbucket2", "dest1",
-        "dest2", "dest3", "dest4", "dest4_sequencefile", "dest_j1", "dest_j2",
-        "dest_g1", "dest_g2", "fetchtask_ioexception",
-        AllVectorTypesRecord.TABLE_NAME}) {
-      db.dropTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, s);
+    if(!isSessionStateStarted) {
+      startSessionState();
     }
+    String cleanupCommands = readEntireFileIntoString(new File(cleanupScript));
+    LOG.info("Cleanup (" + cleanupScript + "):\n" + cleanupCommands);
+    if(cliDriver == null) {
+      cliDriver = new CliDriver();
+    }
+    cliDriver.processLine(cleanupCommands);
 
     // delete any contents in the warehouse dir
     Path p = new Path(testWarehouse);
@@ -653,119 +675,20 @@ public class QTestUtil {
   }
 
   public void createSources() throws Exception {
-
-    startSessionState();
+    if(!isSessionStateStarted) {
+      startSessionState();
+    }
     conf.setBoolean("hive.test.init.phase", true);
 
-    // Create a bunch of tables with columns key and value
-    LinkedList<String> cols = new LinkedList<String>();
-    cols.add("key");
-    cols.add("value");
-
-    LinkedList<String> part_cols = new LinkedList<String>();
-    part_cols.add("ds");
-    part_cols.add("hr");
-    db.createTable("srcpart", cols, part_cols, TextInputFormat.class,
-        IgnoreKeyTextOutputFormat.class);
-
-    Path fpath;
-    HashMap<String, String> part_spec = new HashMap<String, String>();
-    for (String ds : new String[] {"2008-04-08", "2008-04-09"}) {
-      for (String hr : new String[] {"11", "12"}) {
-        part_spec.clear();
-        part_spec.put("ds", ds);
-        part_spec.put("hr", hr);
-        // System.out.println("Loading partition with spec: " + part_spec);
-        // db.createPartition(srcpart, part_spec);
-        fpath = new Path(testFiles, "kv1.txt");
-        // db.loadPartition(fpath, srcpart.getName(), part_spec, true);
-        runLoadCmd("LOAD DATA LOCAL INPATH '" + fpath.toUri().getPath()
-            + "' OVERWRITE INTO TABLE srcpart PARTITION (ds='" + ds + "',hr='"
-            + hr + "')");
-      }
-    }
-    ArrayList<String> bucketCols = new ArrayList<String>();
-    bucketCols.add("key");
-    runCreateTableCmd("CREATE TABLE srcbucket(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE");
-    // db.createTable("srcbucket", cols, null, TextInputFormat.class,
-    // IgnoreKeyTextOutputFormat.class, 2, bucketCols);
-    for (String fname : new String[] {"srcbucket0.txt", "srcbucket1.txt"}) {
-      fpath = new Path(testFiles, fname);
-      runLoadCmd("LOAD DATA LOCAL INPATH '" + fpath.toUri().getPath()
-          + "' INTO TABLE srcbucket");
-    }
-
-    runCreateTableCmd("CREATE TABLE srcbucket2(key int, value string) "
-        + "CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE");
-    // db.createTable("srcbucket", cols, null, TextInputFormat.class,
-    // IgnoreKeyTextOutputFormat.class, 2, bucketCols);
-    for (String fname : new String[] {"srcbucket20.txt", "srcbucket21.txt",
-        "srcbucket22.txt", "srcbucket23.txt"}) {
-      fpath = new Path(testFiles, fname);
-      runLoadCmd("LOAD DATA LOCAL INPATH '" + fpath.toUri().getPath()
-          + "' INTO TABLE srcbucket2");
-    }
-
-    for (String tname : new String[] {"src", "src1"}) {
-      db.createTable(tname, cols, null, TextInputFormat.class,
-          IgnoreKeyTextOutputFormat.class);
-    }
-    db.createTable("src_sequencefile", cols, null,
-        SequenceFileInputFormat.class, SequenceFileOutputFormat.class);
-
-    Table srcThrift =
-        new Table(SessionState.get().getCurrentDatabase(), "src_thrift");
-    srcThrift.setInputFormatClass(SequenceFileInputFormat.class.getName());
-    srcThrift.setOutputFormatClass(SequenceFileOutputFormat.class.getName());
-    srcThrift.setSerializationLib(ThriftDeserializer.class.getName());
-    srcThrift.setSerdeParam(serdeConstants.SERIALIZATION_CLASS, Complex.class
-        .getName());
-    srcThrift.setSerdeParam(serdeConstants.SERIALIZATION_FORMAT,
-        TBinaryProtocol.class.getName());
-    db.createTable(srcThrift);
-
-    LinkedList<String> json_cols = new LinkedList<String>();
-    json_cols.add("json");
-    db.createTable("src_json", json_cols, null, TextInputFormat.class,
-        IgnoreKeyTextOutputFormat.class);
-
-    // load the input data into the src table
-    fpath = new Path(testFiles, "kv1.txt");
-    runLoadCmd("LOAD DATA LOCAL INPATH '" + fpath.toUri().getPath() + "' INTO TABLE src");
-
-    // load the input data into the src table
-    fpath = new Path(testFiles, "kv3.txt");
-    runLoadCmd("LOAD DATA LOCAL INPATH '" + fpath.toUri().getPath() + "' INTO TABLE src1");
-
-    // load the input data into the src_sequencefile table
-    fpath = new Path(testFiles, "kv1.seq");
-    runLoadCmd("LOAD DATA LOCAL INPATH '" + fpath.toUri().getPath()
-        + "' INTO TABLE src_sequencefile");
-
-    // load the input data into the src_thrift table
-    fpath = new Path(testFiles, "complex.seq");
-    runLoadCmd("LOAD DATA LOCAL INPATH '" + fpath.toUri().getPath()
-        + "' INTO TABLE src_thrift");
-
-    // load the json data into the src_json table
-    fpath = new Path(testFiles, "json.txt");
-    runLoadCmd("LOAD DATA LOCAL INPATH '" + fpath.toUri().getPath()
-        + "' INTO TABLE src_json");
-
-    FileSystem localFs = FileSystem.getLocal(conf);
-    // create and load data into orc table
-    fpath = new Path(testFiles, AllVectorTypesRecord.TABLE_NAME);
-
-    runCreateTableCmd(AllVectorTypesRecord.TABLE_CREATE_COMMAND);
-    runLoadCmd("LOAD DATA LOCAL INPATH '" + fpath.toUri().getPath()
-        + "' INTO  TABLE "+AllVectorTypesRecord.TABLE_NAME);
-
-    runCmd("DROP FUNCTION IF EXISTS qtest_get_java_boolean ");
-    runCmd("CREATE FUNCTION qtest_get_java_boolean "
-        + " AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFTestGetJavaBoolean'");
+    String initCommands = readEntireFileIntoString(new File(this.initScript));
+    LOG.info("Initial setup (" + initScript + "):\n" + initCommands);
+    if(cliDriver == null) {
+      cliDriver = new CliDriver();
+    }
+    cliDriver.processLine("set test.data.dir=" + testFiles + ";");
+    cliDriver.processLine(initCommands);
 
     conf.setBoolean("hive.test.init.phase", false);
-
   }
 
   public void init() throws Exception {
@@ -786,33 +709,7 @@ public class QTestUtil {
   public void init(String tname) throws Exception {
     cleanUp();
     createSources();
-
-    LinkedList<String> cols = new LinkedList<String>();
-    cols.add("key");
-    cols.add("value");
-
-    LinkedList<String> part_cols = new LinkedList<String>();
-    part_cols.add("ds");
-    part_cols.add("hr");
-
-    db.createTable("dest1", cols, null, TextInputFormat.class,
-        IgnoreKeyTextOutputFormat.class);
-    db.createTable("dest2", cols, null, TextInputFormat.class,
-        IgnoreKeyTextOutputFormat.class);
-
-    db.createTable("dest3", cols, part_cols, TextInputFormat.class,
-        IgnoreKeyTextOutputFormat.class);
-    Table dest3 = db.getTable("dest3");
-
-    HashMap<String, String> part_spec = new HashMap<String, String>();
-    part_spec.put("ds", "2008-04-08");
-    part_spec.put("hr", "12");
-    db.createPartition(dest3, part_spec);
-
-    db.createTable("dest4", cols, null, TextInputFormat.class,
-        IgnoreKeyTextOutputFormat.class);
-    db.createTable("dest4_sequencefile", cols, null,
-        SequenceFileInputFormat.class, SequenceFileOutputFormat.class);
+    cliDriver.processCmd("set hive.cli.print.header=true;");
   }
 
   public void cliInit(String tname) throws Exception {
@@ -866,23 +763,38 @@ public class QTestUtil {
     SessionState.start(ss);
 
     cliDriver = new CliDriver();
+
     if (tname.equals("init_file.q")) {
       ss.initFiles.add("../../data/scripts/test_init_file.sql");
     }
     cliDriver.processInitFiles(ss);
+
     return outf.getAbsolutePath();
   }
 
   private CliSessionState startSessionState()
-      throws FileNotFoundException, UnsupportedEncodingException {
+      throws IOException {
 
     HiveConf.setVar(conf, HiveConf.ConfVars.HIVE_AUTHENTICATOR_MANAGER,
         "org.apache.hadoop.hive.ql.security.DummyAuthenticator");
 
     CliSessionState ss = new CliSessionState(conf);
     assert ss != null;
+    ss.in = System.in;
+    ss.out = System.out;
+    ss.err = System.out;
 
+    SessionState oldSs = SessionState.get();
+    if (oldSs != null && clusterType == MiniClusterType.tez) {
+      oldSs.close();
+    }
+    if (oldSs != null && oldSs.out != null && oldSs.out != System.out) {
+      oldSs.out.close();
+    }
     SessionState.start(ss);
+
+    isSessionStateStarted = true;
+
     return ss;
   }
 
@@ -1571,7 +1483,7 @@ public class QTestUtil {
   {
     QTestUtil[] qt = new QTestUtil[qfiles.length];
     for (int i = 0; i < qfiles.length; i++) {
-      qt[i] = new QTestUtil(resDir, logDir, MiniClusterType.none, null, "0.20");
+      qt[i] = new QTestUtil(resDir, logDir, MiniClusterType.none, null, "0.20", "", "");
       qt[i].addFile(qfiles[i]);
       qt[i].clearTestSideEffects();
     }

Modified: hive/branches/cbo/itests/util/src/main/java/org/apache/hadoop/hive/ql/hooks/CheckColumnAccessHook.java
URL: http://svn.apache.org/viewvc/hive/branches/cbo/itests/util/src/main/java/org/apache/hadoop/hive/ql/hooks/CheckColumnAccessHook.java?rev=1617652&r1=1617651&r2=1617652&view=diff
==============================================================================
--- hive/branches/cbo/itests/util/src/main/java/org/apache/hadoop/hive/ql/hooks/CheckColumnAccessHook.java (original)
+++ hive/branches/cbo/itests/util/src/main/java/org/apache/hadoop/hive/ql/hooks/CheckColumnAccessHook.java Wed Aug 13 02:28:54 2014
@@ -18,9 +18,9 @@
 package org.apache.hadoop.hive.ql.hooks;
 
 import java.util.Arrays;
+import java.util.List;
 import java.util.Map;
 import java.util.HashMap;
-import java.util.Set;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -34,7 +34,7 @@ import org.mortbay.log.Log;
 /*
  * This hook is used for verifying the column access information
  * that is generated and maintained in the QueryPlan object by the
- * ColumnAccessAnalyer. All the hook does is print out the columns
+ * ColumnAccessAnalyzer. All the hook does is print out the columns
  * accessed from each table as recorded in the ColumnAccessInfo
  * in the QueryPlan.
  */
@@ -58,14 +58,14 @@ public class CheckColumnAccessHook imple
     }
 
     LogHelper console = SessionState.getConsole();
-    Map<String, Set<String>> tableToColumnAccessMap =
+    Map<String, List<String>> tableToColumnAccessMap =
       columnAccessInfo.getTableToColumnAccessMap();
 
     // We need a new map to ensure output is always produced in the same order.
     // This makes tests that use this hook deterministic.
     Map<String, String> outputOrderedMap = new HashMap<String, String>();
 
-    for (Map.Entry<String, Set<String>> tableAccess : tableToColumnAccessMap.entrySet()) {
+    for (Map.Entry<String, List<String>> tableAccess : tableToColumnAccessMap.entrySet()) {
       StringBuilder perTableInfo = new StringBuilder();
       perTableInfo.append("Table:").append(tableAccess.getKey()).append("\n");
       // Sort columns to make output deterministic