Author: gunther
Date: Tue Jul 29 23:15:13 2014
New Revision: 1614514

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

Added:
    hive/branches/cbo/ql/src/test/queries/clientpositive/vector_string_concat.q
      - copied unchanged from r1614505, 
hive/trunk/ql/src/test/queries/clientpositive/vector_string_concat.q
    
hive/branches/cbo/ql/src/test/results/clientpositive/tez/vector_string_concat.q.out
      - copied unchanged from r1614505, 
hive/trunk/ql/src/test/results/clientpositive/tez/vector_string_concat.q.out
    
hive/branches/cbo/ql/src/test/results/clientpositive/vector_string_concat.q.out
      - copied unchanged from r1614505, 
hive/trunk/ql/src/test/results/clientpositive/vector_string_concat.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/itests/qtest/testconfiguration.properties
    
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringConcatColScalar.java
    
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringConcatScalarCol.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/exec/vector/expressions/VectorUDFTimestampFieldLong.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/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/SortedMergeJoinProc.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/HiveParser.g
    
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeConstantDesc.java
    
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
    
hive/branches/cbo/ql/src/test/results/clientpositive/annotate_stats_filter.q.out
    
hive/branches/cbo/ql/src/test/results/clientpositive/annotate_stats_join.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/join_nullsafe.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/metadataonly1.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/smb_mapjoin_25.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/tez/metadataonly1.q.out

Propchange: hive/branches/cbo/
------------------------------------------------------------------------------
  Merged /hive/trunk:r1614212-1614505

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=1614514&r1=1614513&r2=1614514&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 
Tue Jul 29 23:15:13 2014
@@ -745,7 +745,7 @@ public class HiveConf extends Configurat
     // HWI
     HIVEHWILISTENHOST("hive.hwi.listen.host", "0.0.0.0", "This is the host 
address the Hive Web Interface will listen on"),
     HIVEHWILISTENPORT("hive.hwi.listen.port", "9999", "This is the port the 
Hive Web Interface will listen on"),
-    HIVEHWIWARFILE("hive.hwi.war.file", "${system:HWI_WAR_FILE}",
+    HIVEHWIWARFILE("hive.hwi.war.file", "${env:HWI_WAR_FILE}",
         "This sets the path to the HWI war file, relative to ${HIVE_HOME}. "),
 
     HIVEHADOOPMAXMEM("hive.mapred.local.mem", 0, "mapper/reducer memory in 
local mode"),
@@ -1394,10 +1394,9 @@ public class HiveConf extends Configurat
         "which you can then extract a URL from and pass to 
PropertyConfigurator.configure(URL)."),
 
     // Hive global init file location
-    HIVE_GLOBAL_INIT_FILE_LOCATION("hive.global.init.file.location", 
System.getenv("HIVE_CONF_DIR"),
+    HIVE_GLOBAL_INIT_FILE_LOCATION("hive.server2.global.init.file.location", 
"${env:HIVE_CONF_DIR}",
         "The location of HS2 global init file (.hiverc).\n" +
-        "If the property is not set, then HS2 will search for the file in 
$HIVE_CONF_DIR/.\n" +
-        "If the property is set, the value must be a valid path where the init 
file is located."),
+        "If the property is reset, the value must be a valid path where the 
init file is located."),
 
     // prefix used to auto generated column aliases (this should be started 
with '_')
     
HIVE_AUTOGEN_COLUMNALIAS_PREFIX_LABEL("hive.autogen.columnalias.prefix.label", 
"_c",

Modified: hive/branches/cbo/conf/hive-default.xml.template
URL: 
http://svn.apache.org/viewvc/hive/branches/cbo/conf/hive-default.xml.template?rev=1614514&r1=1614513&r2=1614514&view=diff
==============================================================================
--- hive/branches/cbo/conf/hive-default.xml.template (original)
+++ hive/branches/cbo/conf/hive-default.xml.template Tue Jul 29 23:15:13 2014
@@ -1191,7 +1191,7 @@
   </property>
   <property>
     <key>hive.hwi.war.file</key>
-    <value>${system:HWI_WAR_FILE}</value>
+    <value>${env:HWI_WAR_FILE}</value>
     <description>This sets the path to the HWI war file, relative to 
${HIVE_HOME}. </description>
   </property>
   <property>
@@ -2480,12 +2480,11 @@
     </description>
   </property>
   <property>
-    <key>hive.global.init.file.location</key>
-    <value/>
+    <key>hive.server2.global.init.file.location</key>
+    <value>${env:HIVE_CONF_DIR}</value>
     <description>
       The location of HS2 global init file (.hiverc).
-      If the property is not set, then HS2 will search for the file in 
$HIVE_CONF_DIR/.
-      If the property is set, the value must be a valid path where the init 
file is located.
+      If the property is reset, the value must be a valid path where the init 
file is located.
     </description>
   </property>
   <property>

Modified: hive/branches/cbo/itests/qtest/testconfiguration.properties
URL: 
http://svn.apache.org/viewvc/hive/branches/cbo/itests/qtest/testconfiguration.properties?rev=1614514&r1=1614513&r2=1614514&view=diff
==============================================================================
--- hive/branches/cbo/itests/qtest/testconfiguration.properties (original)
+++ hive/branches/cbo/itests/qtest/testconfiguration.properties Tue Jul 29 
23:15:13 2014
@@ -1,5 +1,5 @@
 
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,empty_dir_in_table.q,temp_table_external.q
 
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
 
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,tez_join_hash.q
-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,metadataonly1.q,temp_t
 
able.q,vectorized_ptf.q,optimize_nullscan.q,vector_cast_constant.q,cbo_correctness.q
+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,metadataonly1.q,temp_t
 
able.q,vectorized_ptf.q,optimize_nullscan.q,vector_cast_constant.q,vector_string_concat.q,cbo_correctness.q
 
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_rename.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_overwr
 
ite_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

Modified: 
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringConcatColScalar.java
URL: 
http://svn.apache.org/viewvc/hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringConcatColScalar.java?rev=1614514&r1=1614513&r2=1614514&view=diff
==============================================================================
--- 
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringConcatColScalar.java
 (original)
+++ 
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringConcatColScalar.java
 Tue Jul 29 23:15:13 2014
@@ -45,6 +45,11 @@ public class StringConcatColScalar exten
 
   @Override
   public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
     BytesColumnVector inputColVector = (BytesColumnVector) batch.cols[colNum];
     BytesColumnVector outV = (BytesColumnVector) batch.cols[outputColumn];
     int[] sel = batch.selected;

Modified: 
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringConcatScalarCol.java
URL: 
http://svn.apache.org/viewvc/hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringConcatScalarCol.java?rev=1614514&r1=1614513&r2=1614514&view=diff
==============================================================================
--- 
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringConcatScalarCol.java
 (original)
+++ 
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringConcatScalarCol.java
 Tue Jul 29 23:15:13 2014
@@ -45,6 +45,11 @@ public class StringConcatScalarCol exten
 
   @Override
   public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+        super.evaluateChildren(batch);
+      }
+
     BytesColumnVector inputColVector = (BytesColumnVector) batch.cols[colNum];
     BytesColumnVector outV = (BytesColumnVector) batch.cols[outputColumn];
     int[] sel = batch.selected;

Modified: 
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateString.java
URL: 
http://svn.apache.org/viewvc/hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateString.java?rev=1614514&r1=1614513&r2=1614514&view=diff
==============================================================================
--- 
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateString.java
 (original)
+++ 
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateString.java
 Tue Jul 29 23:15:13 2014
@@ -18,6 +18,9 @@
 
 package org.apache.hadoop.hive.ql.exec.vector.expressions;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.ql.exec.vector.VectorGroupByOperator;
 import org.apache.hadoop.io.Text;
 
 import java.sql.Date;
@@ -25,6 +28,9 @@ import java.sql.Date;
 public class VectorUDFDateString extends StringUnaryUDF {
   private static final long serialVersionUID = 1L;
 
+  private static final Log LOG = LogFactory.getLog(
+      VectorUDFDateString.class.getName());
+
   public VectorUDFDateString(int colNum, int outputColumn) {
     super(colNum, outputColumn, new StringUnaryUDF.IUDFUnaryString() {
       Text t = new Text();
@@ -39,7 +45,9 @@ public class VectorUDFDateString extends
           t.set(date.toString());
           return t;
         } catch (IllegalArgumentException e) {
-          e.printStackTrace();
+          if (LOG.isDebugEnabled()) {
+            LOG.info("VectorUDFDateString passed bad string for Date.valueOf 
'" + s.toString() + "'");
+          }
           return null;
         }
       }

Modified: 
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFTimestampFieldLong.java
URL: 
http://svn.apache.org/viewvc/hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFTimestampFieldLong.java?rev=1614514&r1=1614513&r2=1614514&view=diff
==============================================================================
--- 
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFTimestampFieldLong.java
 (original)
+++ 
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFTimestampFieldLong.java
 Tue Jul 29 23:15:13 2014
@@ -88,6 +88,11 @@ public abstract class VectorUDFTimestamp
 
   @Override
   public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+        super.evaluateChildren(batch);
+      }
+
     LongColumnVector outV = (LongColumnVector) batch.cols[outputColumn];
     LongColumnVector inputCol = (LongColumnVector)batch.cols[this.colNum];
     /* every line below this is identical for evaluateLong & evaluateString */

Modified: 
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/AbstractBucketJoinProc.java
URL: 
http://svn.apache.org/viewvc/hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/AbstractBucketJoinProc.java?rev=1614514&r1=1614513&r2=1614514&view=diff
==============================================================================
--- 
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/AbstractBucketJoinProc.java
 (original)
+++ 
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/AbstractBucketJoinProc.java
 Tue Jul 29 23:15:13 2014
@@ -28,6 +28,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Stack;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileStatus;
@@ -50,6 +51,7 @@ import org.apache.hadoop.hive.ql.parse.Q
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.parse.TableAccessAnalyzer;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.MapJoinDesc;
 import org.apache.hadoop.hive.ql.plan.OperatorDesc;
@@ -456,10 +458,19 @@ abstract public class AbstractBucketJoin
   public static List<String> toColumns(List<ExprNodeDesc> keys) {
     List<String> columns = new ArrayList<String>();
     for (ExprNodeDesc key : keys) {
-      if (!(key instanceof ExprNodeColumnDesc)) {
+      if (key instanceof ExprNodeColumnDesc) {
+        columns.add(((ExprNodeColumnDesc) key).getColumn());
+      } else if ((key instanceof ExprNodeConstantDesc)) {
+        ExprNodeConstantDesc constant = (ExprNodeConstantDesc) key;
+        String colName = constant.getFoldedFromCol();
+        if (colName == null){
+          return null;
+        } else {
+          columns.add(colName);
+        }
+      } else {
         return null;
       }
-      columns.add(((ExprNodeColumnDesc) key).getColumn());
     }
     return columns;
   }

Modified: 
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagate.java
URL: 
http://svn.apache.org/viewvc/hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagate.java?rev=1614514&r1=1614513&r2=1614514&view=diff
==============================================================================
--- 
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagate.java
 (original)
+++ 
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagate.java
 Tue Jul 29 23:15:13 2014
@@ -82,9 +82,7 @@ public class ConstantPropagate implement
       //    if the later is enabled.
       return pactx;
     }
-    if (pactx.getConf().getBoolVar(ConfVars.HIVEOPTSORTMERGEBUCKETMAPJOIN)) {
-      return pactx;
-    }
+
     pGraphContext = pactx;
     opToParseCtxMap = pGraphContext.getOpParseCtx();
 

Modified: 
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java
URL: 
http://svn.apache.org/viewvc/hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java?rev=1614514&r1=1614513&r2=1614514&view=diff
==============================================================================
--- 
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java
 (original)
+++ 
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java
 Tue Jul 29 23:15:13 2014
@@ -552,6 +552,7 @@ public final class ConstantPropagateProc
    * conditional expressions and extract assignment expressions and propagate 
them.
    */
   public static class ConstantPropagateFilterProc implements NodeProcessor {
+    @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx ctx, 
Object... nodeOutputs)
         throws SemanticException {
       FilterOperator op = (FilterOperator) nd;
@@ -594,6 +595,7 @@ public final class ConstantPropagateProc
    * Node Processor for Constant Propagate for Group By Operators.
    */
   public static class ConstantPropagateGroupByProc implements NodeProcessor {
+    @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx ctx, 
Object... nodeOutputs)
         throws SemanticException {
       GroupByOperator op = (GroupByOperator) nd;
@@ -630,6 +632,7 @@ public final class ConstantPropagateProc
    * The Default Node Processor for Constant Propagation.
    */
   public static class ConstantPropagateDefaultProc implements NodeProcessor {
+    @Override
     @SuppressWarnings("unchecked")
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx ctx, 
Object... nodeOutputs)
         throws SemanticException {
@@ -658,6 +661,7 @@ public final class ConstantPropagateProc
    * The Node Processor for Constant Propagation for Select Operators.
    */
   public static class ConstantPropagateSelectProc implements NodeProcessor {
+    @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx ctx, 
Object... nodeOutputs)
         throws SemanticException {
       SelectOperator op = (SelectOperator) nd;
@@ -691,6 +695,7 @@ public final class ConstantPropagateProc
    * propagation, this processor also prunes dynamic partitions to static 
partitions if possible.
    */
   public static class ConstantPropagateFileSinkProc implements NodeProcessor {
+    @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx ctx, 
Object... nodeOutputs)
         throws SemanticException {
       FileSinkOperator op = (FileSinkOperator) nd;
@@ -743,6 +748,7 @@ public final class ConstantPropagateProc
    * Currently these kinds of Operators include UnionOperator and 
ScriptOperator.
    */
   public static class ConstantPropagateStopProc implements NodeProcessor {
+    @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx ctx, 
Object... nodeOutputs)
         throws SemanticException {
       Operator<?> op = (Operator<?>) nd;
@@ -763,6 +769,7 @@ public final class ConstantPropagateProc
    * join (left table for left outer join and vice versa) can be propagated.
    */
   public static class ConstantPropagateReduceSinkProc implements NodeProcessor 
{
+    @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx ctx, 
Object... nodeOutputs)
         throws SemanticException {
       ReduceSinkOperator op = (ReduceSinkOperator) nd;
@@ -795,7 +802,11 @@ public final class ConstantPropagateProc
       // key columns
       ArrayList<ExprNodeDesc> newKeyEpxrs = new ArrayList<ExprNodeDesc>();
       for (ExprNodeDesc desc : rsDesc.getKeyCols()) {
-        newKeyEpxrs.add(foldExpr(desc, constants, cppCtx, op, 0, false));
+        ExprNodeDesc newDesc = foldExpr(desc, constants, cppCtx, op, 0, false);
+        if (newDesc != desc && desc instanceof ExprNodeColumnDesc && newDesc 
instanceof ExprNodeConstantDesc) {
+          
((ExprNodeConstantDesc)newDesc).setFoldedFromCol(((ExprNodeColumnDesc)desc).getColumn());
+        }
+        newKeyEpxrs.add(newDesc);
       }
       rsDesc.setKeyCols(newKeyEpxrs);
 
@@ -854,6 +865,7 @@ public final class ConstantPropagateProc
    * The Node Processor for Constant Propagation for Join Operators.
    */
   public static class ConstantPropagateJoinProc implements NodeProcessor {
+    @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx ctx, 
Object... nodeOutputs)
         throws SemanticException {
       JoinOperator op = (JoinOperator) nd;
@@ -916,6 +928,7 @@ public final class ConstantPropagateProc
    * The Node Processor for Constant Propagation for Table Scan Operators.
    */
   public static class ConstantPropagateTableScanProc implements NodeProcessor {
+    @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx ctx, 
Object... nodeOutputs)
         throws SemanticException {
       TableScanOperator op = (TableScanOperator) nd;

Modified: 
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedMergeJoinProc.java
URL: 
http://svn.apache.org/viewvc/hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedMergeJoinProc.java?rev=1614514&r1=1614513&r2=1614514&view=diff
==============================================================================
--- 
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedMergeJoinProc.java
 (original)
+++ 
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedMergeJoinProc.java
 Tue Jul 29 23:15:13 2014
@@ -18,17 +18,13 @@
 
 package org.apache.hadoop.hive.ql.optimizer;
 
-import java.util.HashMap;
-import java.util.Map;
 import java.util.Stack;
 
 import org.apache.hadoop.hive.ql.exec.JoinOperator;
-import org.apache.hadoop.hive.ql.exec.MapJoinOperator;
 import org.apache.hadoop.hive.ql.lib.Node;
 import org.apache.hadoop.hive.ql.lib.NodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
-import org.apache.hadoop.hive.ql.parse.QBJoinTree;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 
 public class SortedMergeJoinProc extends AbstractSMBJoinProc implements 
NodeProcessor {
@@ -46,12 +42,6 @@ public class SortedMergeJoinProc extends
 
     JoinOperator joinOp = (JoinOperator) nd;
     SortBucketJoinProcCtx smbJoinContext = (SortBucketJoinProcCtx) procCtx;
-    Map<MapJoinOperator, QBJoinTree> mapJoinMap = 
pGraphContext.getMapJoinContext();
-    if (mapJoinMap == null) {
-      mapJoinMap = new HashMap<MapJoinOperator, QBJoinTree>();
-      pGraphContext.setMapJoinContext(mapJoinMap);
-    }
-
     boolean convert =
         canConvertJoinToSMBJoin(
             joinOp, smbJoinContext, pGraphContext);

Modified: 
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java
URL: 
http://svn.apache.org/viewvc/hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java?rev=1614514&r1=1614513&r2=1614514&view=diff
==============================================================================
--- 
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java
 (original)
+++ 
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java
 Tue Jul 29 23:15:13 2014
@@ -401,7 +401,6 @@ public class StatsRulesProcFactory {
 
       long numRows = stats.getNumRows();
 
-      // evaluate similar to "col = constant" expr
       if (pred instanceof ExprNodeGenericFuncDesc) {
 
         ExprNodeGenericFuncDesc genFunc = (ExprNodeGenericFuncDesc) pred;
@@ -413,9 +412,7 @@ public class StatsRulesProcFactory {
             String tabAlias = colDesc.getTabAlias();
             ColStatistics cs = stats.getColumnStatisticsForColumn(tabAlias, 
colName);
             if (cs != null) {
-              long dvs = cs.getCountDistint();
-              numRows = dvs == 0 ? numRows / 2 : numRows / dvs;
-              return numRows;
+              return cs.getNumNulls();
             }
           }
         }

Modified: 
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
URL: 
http://svn.apache.org/viewvc/hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g?rev=1614514&r1=1614513&r2=1614514&view=diff
==============================================================================
--- hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g 
(original)
+++ hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g 
Tue Jul 29 23:15:13 2014
@@ -928,7 +928,7 @@ alterStatement
         |
             KW_INDEX! alterIndexStatementSuffix
         |
-            KW_DATABASE! alterDatabaseStatementSuffix
+            (KW_DATABASE|KW_SCHEMA)! alterDatabaseStatementSuffix
         )
     ;
 
@@ -1299,7 +1299,7 @@ descStatement
 @after { popMsg(state); }
     : (KW_DESCRIBE|KW_DESC) 
(descOptions=KW_FORMATTED|descOptions=KW_EXTENDED|descOptions=KW_PRETTY)? 
(parttype=descPartTypeExpr) -> ^(TOK_DESCTABLE $parttype $descOptions?)
     | (KW_DESCRIBE|KW_DESC) KW_FUNCTION KW_EXTENDED? (name=descFuncNames) -> 
^(TOK_DESCFUNCTION $name KW_EXTENDED?)
-    | (KW_DESCRIBE|KW_DESC) KW_DATABASE KW_EXTENDED? (dbName=identifier) -> 
^(TOK_DESCDATABASE $dbName KW_EXTENDED?)
+    | (KW_DESCRIBE|KW_DESC) (KW_DATABASE|KW_SCHEMA) KW_EXTENDED? 
(dbName=identifier) -> ^(TOK_DESCDATABASE $dbName KW_EXTENDED?)
     ;
 
 analyzeStatement
@@ -1324,7 +1324,7 @@ showStatement
     -> ^(TOK_SHOW_TABLESTATUS showStmtIdentifier $db_name? partitionSpec?)
     | KW_SHOW KW_TBLPROPERTIES tblName=identifier (LPAREN 
prptyName=StringLiteral RPAREN)? -> ^(TOK_SHOW_TBLPROPERTIES $tblName 
$prptyName?)
     | KW_SHOW KW_LOCKS (parttype=partTypeExpr)? (isExtended=KW_EXTENDED)? -> 
^(TOK_SHOWLOCKS $parttype? $isExtended?)
-    | KW_SHOW KW_LOCKS KW_DATABASE (dbName=Identifier) 
(isExtended=KW_EXTENDED)? -> ^(TOK_SHOWDBLOCKS $dbName $isExtended?)
+    | KW_SHOW KW_LOCKS (KW_DATABASE|KW_SCHEMA) (dbName=Identifier) 
(isExtended=KW_EXTENDED)? -> ^(TOK_SHOWDBLOCKS $dbName $isExtended?)
     | KW_SHOW (showOptions=KW_FORMATTED)? (KW_INDEX|KW_INDEXES) KW_ON 
showStmtIdentifier ((KW_FROM|KW_IN) db_name=identifier)?
     -> ^(TOK_SHOWINDEXES showStmtIdentifier $showOptions? $db_name?)
     | KW_SHOW KW_COMPACTIONS -> ^(TOK_SHOW_COMPACTIONS)
@@ -1341,7 +1341,7 @@ lockStatement
 lockDatabase
 @init { pushMsg("lock database statement", state); }
 @after { popMsg(state); }
-    : KW_LOCK KW_DATABASE (dbName=Identifier) lockMode -> ^(TOK_LOCKDB $dbName 
lockMode)
+    : KW_LOCK (KW_DATABASE|KW_SCHEMA) (dbName=Identifier) lockMode -> 
^(TOK_LOCKDB $dbName lockMode)
     ;
 
 lockMode
@@ -1359,7 +1359,7 @@ unlockStatement
 unlockDatabase
 @init { pushMsg("unlock database statement", state); }
 @after { popMsg(state); }
-    : KW_UNLOCK KW_DATABASE (dbName=Identifier) -> ^(TOK_UNLOCKDB $dbName)
+    : KW_UNLOCK (KW_DATABASE|KW_SCHEMA) (dbName=Identifier) -> ^(TOK_UNLOCKDB 
$dbName)
     ;
 
 createRoleStatement
@@ -1471,7 +1471,7 @@ privilegeObject
 privObjectType
 @init {pushMsg("privilege object type type", state);}
 @after {popMsg(state);}
-    : KW_DATABASE -> ^(TOK_DB_TYPE)
+    : (KW_DATABASE|KW_SCHEMA) -> ^(TOK_DB_TYPE)
     | KW_TABLE? -> ^(TOK_TABLE_TYPE)
     ;
 

Modified: 
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeConstantDesc.java
URL: 
http://svn.apache.org/viewvc/hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeConstantDesc.java?rev=1614514&r1=1614513&r2=1614514&view=diff
==============================================================================
--- 
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeConstantDesc.java
 (original)
+++ 
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeConstantDesc.java
 Tue Jul 29 23:15:13 2014
@@ -36,6 +36,17 @@ public class ExprNodeConstantDesc extend
   private static final long serialVersionUID = 1L;
   final protected transient static char[] hexArray = 
"0123456789ABCDEF".toCharArray();
   private Object value;
+  // If this constant was created while doing constant folding, foldedFromCol 
holds the name of
+  // original column from which it was folded.
+  private transient String foldedFromCol;
+
+  public String getFoldedFromCol() {
+    return foldedFromCol;
+  }
+
+  public void setFoldedFromCol(String foldedFromCol) {
+    this.foldedFromCol = foldedFromCol;
+  }
 
   public ExprNodeConstantDesc() {
   }

Modified: 
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
URL: 
http://svn.apache.org/viewvc/hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java?rev=1614514&r1=1614513&r2=1614514&view=diff
==============================================================================
--- 
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java 
(original)
+++ 
hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java 
Tue Jul 29 23:15:13 2014
@@ -437,6 +437,8 @@ public class StatsUtils {
       cs.setAvgColLen(JavaDataModel.get().lengthOfTimestamp());
     } else if (colType.startsWith(serdeConstants.DECIMAL_TYPE_NAME)) {
       cs.setAvgColLen(JavaDataModel.get().lengthOfDecimal());
+      cs.setCountDistint(csd.getDecimalStats().getNumDVs());
+      cs.setNumNulls(csd.getDecimalStats().getNumNulls());
     } else if (colType.equalsIgnoreCase(serdeConstants.DATE_TYPE_NAME)) {
       cs.setAvgColLen(JavaDataModel.get().lengthOfDate());
     } else {

Modified: 
hive/branches/cbo/ql/src/test/results/clientpositive/annotate_stats_filter.q.out
URL: 
http://svn.apache.org/viewvc/hive/branches/cbo/ql/src/test/results/clientpositive/annotate_stats_filter.q.out?rev=1614514&r1=1614513&r2=1614514&view=diff
==============================================================================
Files 
hive/branches/cbo/ql/src/test/results/clientpositive/annotate_stats_filter.q.out
 (original) and 
hive/branches/cbo/ql/src/test/results/clientpositive/annotate_stats_filter.q.out
 Tue Jul 29 23:15:13 2014 differ

Modified: 
hive/branches/cbo/ql/src/test/results/clientpositive/annotate_stats_join.q.out
URL: 
http://svn.apache.org/viewvc/hive/branches/cbo/ql/src/test/results/clientpositive/annotate_stats_join.q.out?rev=1614514&r1=1614513&r2=1614514&view=diff
==============================================================================
Files 
hive/branches/cbo/ql/src/test/results/clientpositive/annotate_stats_join.q.out 
(original) and 
hive/branches/cbo/ql/src/test/results/clientpositive/annotate_stats_join.q.out 
Tue Jul 29 23:15:13 2014 differ

Modified: 
hive/branches/cbo/ql/src/test/results/clientpositive/join_nullsafe.q.out
URL: 
http://svn.apache.org/viewvc/hive/branches/cbo/ql/src/test/results/clientpositive/join_nullsafe.q.out?rev=1614514&r1=1614513&r2=1614514&view=diff
==============================================================================
--- hive/branches/cbo/ql/src/test/results/clientpositive/join_nullsafe.q.out 
(original)
+++ hive/branches/cbo/ql/src/test/results/clientpositive/join_nullsafe.q.out 
Tue Jul 29 23:15:13 2014
@@ -1519,9 +1519,8 @@ STAGE PLANS:
               predicate: value is null (type: boolean)
               Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE 
Column stats: NONE
               Reduce Output Operator
-                key expressions: value (type: int)
+                key expressions: null (type: void)
                 sort order: +
-                Map-reduce partition columns: value (type: int)
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE 
Column stats: NONE
                 value expressions: key (type: int)
           TableScan
@@ -1531,9 +1530,8 @@ STAGE PLANS:
               predicate: key is null (type: boolean)
               Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE 
Column stats: NONE
               Reduce Output Operator
-                key expressions: key (type: int)
+                key expressions: null (type: void)
                 sort order: +
-                Map-reduce partition columns: key (type: int)
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE 
Column stats: NONE
                 value expressions: value (type: int)
       Reduce Operator Tree:
@@ -1541,13 +1539,13 @@ STAGE PLANS:
           condition map:
                Inner Join 0 to 1
           condition expressions:
-            0 {KEY.reducesinkkey0} {VALUE._col0}
-            1 {VALUE._col0} {KEY.reducesinkkey0}
+            0 {VALUE._col0}
+            1 {VALUE._col0}
           nullSafes: [true]
-          outputColumnNames: _col0, _col1, _col4, _col5
+          outputColumnNames: _col1, _col4
           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column 
stats: NONE
           Select Operator
-            expressions: _col0 (type: int), _col1 (type: int), _col4 (type: 
int), _col5 (type: int)
+            expressions: null (type: void), _col1 (type: int), _col4 (type: 
int), null (type: void)
             outputColumnNames: _col0, _col1, _col2, _col3
             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column 
stats: NONE
             File Output Operator

Modified: 
hive/branches/cbo/ql/src/test/results/clientpositive/metadataonly1.q.out
URL: 
http://svn.apache.org/viewvc/hive/branches/cbo/ql/src/test/results/clientpositive/metadataonly1.q.out?rev=1614514&r1=1614513&r2=1614514&view=diff
==============================================================================
Files hive/branches/cbo/ql/src/test/results/clientpositive/metadataonly1.q.out 
(original) and 
hive/branches/cbo/ql/src/test/results/clientpositive/metadataonly1.q.out Tue 
Jul 29 23:15:13 2014 differ

Modified: 
hive/branches/cbo/ql/src/test/results/clientpositive/smb_mapjoin_25.q.out
URL: 
http://svn.apache.org/viewvc/hive/branches/cbo/ql/src/test/results/clientpositive/smb_mapjoin_25.q.out?rev=1614514&r1=1614513&r2=1614514&view=diff
==============================================================================
--- hive/branches/cbo/ql/src/test/results/clientpositive/smb_mapjoin_25.q.out 
(original)
+++ hive/branches/cbo/ql/src/test/results/clientpositive/smb_mapjoin_25.q.out 
Tue Jul 29 23:15:13 2014
@@ -219,14 +219,13 @@ STAGE PLANS:
                 condition map:
                      Inner Join 0 to 1
                 condition expressions:
-                  0 {key}
+                  0 
                   1 
                 keys:
-                  0 key (type: int)
-                  1 key (type: int)
-                outputColumnNames: _col0
+                  0 5 (type: int)
+                  1 5 (type: int)
                 Select Operator
-                  expressions: _col0 (type: int)
+                  expressions: 5 (type: int)
                   outputColumnNames: _col0
                   Reduce Output Operator
                     key expressions: _col0 (type: int)
@@ -242,14 +241,13 @@ STAGE PLANS:
                 condition map:
                      Inner Join 0 to 1
                 condition expressions:
-                  0 {key}
+                  0 
                   1 
                 keys:
-                  0 key (type: int)
-                  1 key (type: int)
-                outputColumnNames: _col0
+                  0 5 (type: int)
+                  1 5 (type: int)
                 Select Operator
-                  expressions: _col0 (type: int)
+                  expressions: 5 (type: int)
                   outputColumnNames: _col0
                   Reduce Output Operator
                     key expressions: _col0 (type: int)
@@ -266,7 +264,7 @@ STAGE PLANS:
           Filter Operator
             predicate: (_col1 = 5) (type: boolean)
             Select Operator
-              expressions: _col0 (type: int), _col1 (type: int)
+              expressions: _col0 (type: int), 5 (type: int)
               outputColumnNames: _col0, _col1
               File Output Operator
                 compressed: false

Modified: 
hive/branches/cbo/ql/src/test/results/clientpositive/tez/metadataonly1.q.out
URL: 
http://svn.apache.org/viewvc/hive/branches/cbo/ql/src/test/results/clientpositive/tez/metadataonly1.q.out?rev=1614514&r1=1614513&r2=1614514&view=diff
==============================================================================
Files 
hive/branches/cbo/ql/src/test/results/clientpositive/tez/metadataonly1.q.out 
(original) and 
hive/branches/cbo/ql/src/test/results/clientpositive/tez/metadataonly1.q.out 
Tue Jul 29 23:15:13 2014 differ


Reply via email to