You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by rh...@apache.org on 2013/11/25 20:15:56 UTC

svn commit: r1545372 [1/7] - in /hive/trunk/ql/src: java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/ java/org/apache/hadoop/hive/ql/plan/ java/org/apache/hadoop/hive/ql/stats/ test/queries/clientpositive/ test/results/clientpositive/

Author: rhbutani
Date: Mon Nov 25 19:15:54 2013
New Revision: 1545372

URL: http://svn.apache.org/r1545372
Log:
HIVE-5849 Improve the stats of operators based on heuristics in the absence of any column statistics (Prasanth Jayachandran via Harish Butani)

Modified:
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/Statistics.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
    hive/trunk/ql/src/test/queries/clientpositive/annotate_stats_groupby.q
    hive/trunk/ql/src/test/results/clientpositive/alter_partition_coltype.q.out
    hive/trunk/ql/src/test/results/clientpositive/annotate_stats_filter.q.out
    hive/trunk/ql/src/test/results/clientpositive/annotate_stats_groupby.q.out
    hive/trunk/ql/src/test/results/clientpositive/annotate_stats_join.q.out
    hive/trunk/ql/src/test/results/clientpositive/annotate_stats_limit.q.out
    hive/trunk/ql/src/test/results/clientpositive/annotate_stats_part.q.out
    hive/trunk/ql/src/test/results/clientpositive/annotate_stats_select.q.out
    hive/trunk/ql/src/test/results/clientpositive/auto_join_reordering_values.q.out
    hive/trunk/ql/src/test/results/clientpositive/auto_sortmerge_join_11.q.out
    hive/trunk/ql/src/test/results/clientpositive/auto_sortmerge_join_12.q.out
    hive/trunk/ql/src/test/results/clientpositive/bucket_map_join_1.q.out
    hive/trunk/ql/src/test/results/clientpositive/bucket_map_join_2.q.out
    hive/trunk/ql/src/test/results/clientpositive/bucketcontext_1.q.out
    hive/trunk/ql/src/test/results/clientpositive/bucketcontext_2.q.out
    hive/trunk/ql/src/test/results/clientpositive/bucketcontext_3.q.out
    hive/trunk/ql/src/test/results/clientpositive/bucketcontext_4.q.out
    hive/trunk/ql/src/test/results/clientpositive/bucketcontext_5.q.out
    hive/trunk/ql/src/test/results/clientpositive/bucketcontext_6.q.out
    hive/trunk/ql/src/test/results/clientpositive/bucketcontext_7.q.out
    hive/trunk/ql/src/test/results/clientpositive/bucketcontext_8.q.out
    hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin1.q.out
    hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin10.q.out
    hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin11.q.out
    hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin12.q.out
    hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin13.q.out
    hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin2.q.out
    hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin3.q.out
    hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin4.q.out
    hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin5.q.out
    hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin7.q.out
    hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin8.q.out
    hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin9.q.out
    hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin_negative.q.out
    hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin_negative2.q.out
    hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin_negative3.q.out
    hive/trunk/ql/src/test/results/clientpositive/filter_join_breaktask.q.out
    hive/trunk/ql/src/test/results/clientpositive/groupby_map_ppr.q.out
    hive/trunk/ql/src/test/results/clientpositive/groupby_map_ppr_multi_distinct.q.out
    hive/trunk/ql/src/test/results/clientpositive/groupby_ppr.q.out
    hive/trunk/ql/src/test/results/clientpositive/groupby_ppr_multi_distinct.q.out
    hive/trunk/ql/src/test/results/clientpositive/groupby_sort_1.q.out
    hive/trunk/ql/src/test/results/clientpositive/groupby_sort_6.q.out
    hive/trunk/ql/src/test/results/clientpositive/groupby_sort_skew_1.q.out
    hive/trunk/ql/src/test/results/clientpositive/input23.q.out
    hive/trunk/ql/src/test/results/clientpositive/input42.q.out
    hive/trunk/ql/src/test/results/clientpositive/input_part1.q.out
    hive/trunk/ql/src/test/results/clientpositive/input_part2.q.out
    hive/trunk/ql/src/test/results/clientpositive/input_part7.q.out
    hive/trunk/ql/src/test/results/clientpositive/input_part9.q.out
    hive/trunk/ql/src/test/results/clientpositive/join17.q.out
    hive/trunk/ql/src/test/results/clientpositive/join26.q.out
    hive/trunk/ql/src/test/results/clientpositive/join32.q.out
    hive/trunk/ql/src/test/results/clientpositive/join32_lessSize.q.out
    hive/trunk/ql/src/test/results/clientpositive/join33.q.out
    hive/trunk/ql/src/test/results/clientpositive/join34.q.out
    hive/trunk/ql/src/test/results/clientpositive/join35.q.out
    hive/trunk/ql/src/test/results/clientpositive/join9.q.out
    hive/trunk/ql/src/test/results/clientpositive/join_filters_overlap.q.out
    hive/trunk/ql/src/test/results/clientpositive/join_map_ppr.q.out
    hive/trunk/ql/src/test/results/clientpositive/load_dyn_part8.q.out
    hive/trunk/ql/src/test/results/clientpositive/louter_join_ppr.q.out
    hive/trunk/ql/src/test/results/clientpositive/metadataonly1.q.out
    hive/trunk/ql/src/test/results/clientpositive/outer_join_ppr.q.out
    hive/trunk/ql/src/test/results/clientpositive/pcr.q.out
    hive/trunk/ql/src/test/results/clientpositive/ppd_join_filter.q.out
    hive/trunk/ql/src/test/results/clientpositive/ppd_union_view.q.out
    hive/trunk/ql/src/test/results/clientpositive/ppd_vc.q.out
    hive/trunk/ql/src/test/results/clientpositive/ppr_allchildsarenull.q.out
    hive/trunk/ql/src/test/results/clientpositive/push_or.q.out
    hive/trunk/ql/src/test/results/clientpositive/rand_partitionpruner1.q.out
    hive/trunk/ql/src/test/results/clientpositive/rand_partitionpruner2.q.out
    hive/trunk/ql/src/test/results/clientpositive/rand_partitionpruner3.q.out
    hive/trunk/ql/src/test/results/clientpositive/regexp_extract.q.out
    hive/trunk/ql/src/test/results/clientpositive/router_join_ppr.q.out
    hive/trunk/ql/src/test/results/clientpositive/sample1.q.out
    hive/trunk/ql/src/test/results/clientpositive/sample10.q.out
    hive/trunk/ql/src/test/results/clientpositive/sample2.q.out
    hive/trunk/ql/src/test/results/clientpositive/sample4.q.out
    hive/trunk/ql/src/test/results/clientpositive/sample5.q.out
    hive/trunk/ql/src/test/results/clientpositive/sample6.q.out
    hive/trunk/ql/src/test/results/clientpositive/sample7.q.out
    hive/trunk/ql/src/test/results/clientpositive/sample8.q.out
    hive/trunk/ql/src/test/results/clientpositive/sample9.q.out
    hive/trunk/ql/src/test/results/clientpositive/smb_mapjoin_13.q.out
    hive/trunk/ql/src/test/results/clientpositive/smb_mapjoin_15.q.out
    hive/trunk/ql/src/test/results/clientpositive/sort_merge_join_desc_6.q.out
    hive/trunk/ql/src/test/results/clientpositive/sort_merge_join_desc_7.q.out
    hive/trunk/ql/src/test/results/clientpositive/stats11.q.out
    hive/trunk/ql/src/test/results/clientpositive/transform_ppr1.q.out
    hive/trunk/ql/src/test/results/clientpositive/transform_ppr2.q.out
    hive/trunk/ql/src/test/results/clientpositive/udf_explode.q.out
    hive/trunk/ql/src/test/results/clientpositive/udtf_explode.q.out
    hive/trunk/ql/src/test/results/clientpositive/union22.q.out
    hive/trunk/ql/src/test/results/clientpositive/union24.q.out
    hive/trunk/ql/src/test/results/clientpositive/union_ppr.q.out

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java?rev=1545372&r1=1545371&r2=1545372&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java Mon Nov 25 19:15:54 2013
@@ -53,6 +53,7 @@ import org.apache.hadoop.hive.ql.plan.Ex
 import org.apache.hadoop.hive.ql.plan.JoinDesc;
 import org.apache.hadoop.hive.ql.plan.OperatorDesc;
 import org.apache.hadoop.hive.ql.plan.Statistics;
+import org.apache.hadoop.hive.ql.plan.Statistics.State;
 import org.apache.hadoop.hive.ql.stats.StatsUtils;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPAnd;
@@ -213,6 +214,9 @@ public class StatsRulesProcFactory {
    * satisfy condition2
    * </ul>
    * <p>
+   * <i>Worst case:</i> If no column statistics are available, then T(R) = T(R)/2 will be
+   * used as heuristics.
+   * <p>
    * <i>For more information, refer 'Estimating The Cost Of Operations' chapter in
    * "Database Systems: The Complete Book" by Garcia-Molina et. al.</i>
    * </p>
@@ -239,7 +243,10 @@ public class StatsRulesProcFactory {
           fop.setStatistics(st);
         } else {
           if (parentStats != null) {
-            fop.setStatistics(parentStats.clone());
+
+            // worst case, in the absence of column statistics assume half the rows are emitted
+            Statistics wcStats = getWorstCaseStats(parentStats.clone());
+            fop.setStatistics(wcStats);
           }
         }
 
@@ -510,6 +517,9 @@ public class StatsRulesProcFactory {
    * assumed.
    *
    * <p>
+   * <i>Worst case:</i> If no column statistics are available, then T(R) = T(R)/2 will be
+   * used as heuristics.
+   * <p>
    * <i>For more information, refer 'Estimating The Cost Of Operations' chapter in
    * "Database Systems: The Complete Book" by Garcia-Molina et. al.</i>
    * </p>
@@ -527,13 +537,15 @@ public class StatsRulesProcFactory {
       HiveConf conf = aspCtx.getConf();
       int mapSideParallelism = HiveConf.getIntVar(conf,
           HiveConf.ConfVars.HIVE_STATS_MAP_SIDE_PARALLELISM);
+      List<AggregationDesc> aggDesc = gop.getConf().getAggregators();
+      Map<String, ExprNodeDesc> colExprMap = gop.getColumnExprMap();
+      RowSchema rs = gop.getSchema();
+      Statistics stats = null;
 
       try {
         if (satisfyPrecondition(parentStats)) {
-          Statistics stats = parentStats.clone();
-          RowSchema rs = gop.getSchema();
-          List<AggregationDesc> aggDesc = gop.getConf().getAggregators();
-          Map<String, ExprNodeDesc> colExprMap = gop.getColumnExprMap();
+          stats = parentStats.clone();
+
           List<ColStatistics> colStats = StatsUtils.getColStatisticsFromExprMap(conf, parentStats,
               colExprMap, rs);
           stats.setColumnStats(colStats);
@@ -548,6 +560,13 @@ public class StatsRulesProcFactory {
                 dv += 1;
               }
               dvProd *= dv;
+            } else {
+
+              // partial column statistics on grouping attributes case.
+              // if column statistics on grouping attribute is missing, then assume worst case.
+              // GBY rule will emit half the number of rows if dvProd is 0
+              dvProd = 0;
+              break;
             }
           }
 
@@ -588,44 +607,54 @@ public class StatsRulesProcFactory {
             newNumRows = applyGBYRule(stats.getNumRows(), dvProd);
             updateStats(stats, newNumRows);
           }
+        } else {
+          if (parentStats != null) {
+
+            // worst case, in the absence of column statistics assume half the rows are emitted
+            if (gop.getChildOperators().get(0) instanceof ReduceSinkOperator) {
+
+              // map side
+              stats = parentStats.clone();
+            } else {
+
+              // reduce side
+              stats = getWorstCaseStats(parentStats);
+            }
+          }
+        }
 
-          // if UDAFs are present, new columns needs to be added
-          if (!aggDesc.isEmpty()) {
-            List<ColStatistics> aggColStats = Lists.newArrayList();
-            for (ColumnInfo ci : rs.getSignature()) {
-
-              // if the columns in row schema is not contained in column
-              // expression map, then those are the aggregate columns that
-              // are added GBY operator. we will estimate the column statistics
-              // for those newly added columns
-              if (!colExprMap.containsKey(ci.getInternalName())) {
-                String colName = ci.getInternalName();
-                colName = StatsUtils.stripPrefixFromColumnName(colName);
-                String tabAlias = ci.getTabAlias();
-                String colType = ci.getTypeName();
-                ColStatistics cs = new ColStatistics(tabAlias, colName, colType);
-                cs.setCountDistint(stats.getNumRows());
-                cs.setNumNulls(0);
-                cs.setAvgColLen(StatsUtils.getAvgColLenOfFixedLengthTypes(colType));
-                aggColStats.add(cs);
-              }
-            }
-            stats.addToColumnStats(aggColStats);
-
-            // if UDAF present and if column expression map is empty then it must
-            // be full aggregation query like count(*) in which case number of rows will be 1
-            if (colExprMap.isEmpty()) {
-              stats.setNumRows(1);
-              updateStats(stats, 1);
+        // if UDAFs are present, new columns needs to be added
+        if (!aggDesc.isEmpty() && stats != null) {
+          List<ColStatistics> aggColStats = Lists.newArrayList();
+          for (ColumnInfo ci : rs.getSignature()) {
+
+            // if the columns in row schema is not contained in column
+            // expression map, then those are the aggregate columns that
+            // are added GBY operator. we will estimate the column statistics
+            // for those newly added columns
+            if (!colExprMap.containsKey(ci.getInternalName())) {
+              String colName = ci.getInternalName();
+              colName = StatsUtils.stripPrefixFromColumnName(colName);
+              String tabAlias = ci.getTabAlias();
+              String colType = ci.getTypeName();
+              ColStatistics cs = new ColStatistics(tabAlias, colName, colType);
+              cs.setCountDistint(stats.getNumRows());
+              cs.setNumNulls(0);
+              cs.setAvgColLen(StatsUtils.getAvgColLenOfFixedLengthTypes(colType));
+              aggColStats.add(cs);
             }
           }
+          stats.addToColumnStats(aggColStats);
 
-          gop.setStatistics(stats);
-        } else {
-          if (parentStats != null) {
-            gop.setStatistics(parentStats.clone());
+          // if UDAF present and if column expression map is empty then it must
+          // be full aggregation query like count(*) in which case number of rows will be 1
+          if (colExprMap.isEmpty()) {
+            stats.setNumRows(1);
+            updateStats(stats, 1);
           }
         }
+
+        gop.setStatistics(stats);
       } catch (CloneNotSupportedException e) {
         throw new SemanticException(ErrorMsg.STATISTICS_CLONING_FAILED.getMsg());
       }
@@ -668,6 +697,9 @@ public class StatsRulesProcFactory {
    * attributes
    *
    * <p>
+   * <i>Worst case:</i> If no column statistics are available, then T(RXS) = T(R)*T(S)/2 will be
+   * used as heuristics.
+   * <p>
    * <i>For more information, refer 'Estimating The Cost Of Operations' chapter in
    * "Database Systems: The Complete Book" by Garcia-Molina et. al.</i>
    * </p>
@@ -698,7 +730,9 @@ public class StatsRulesProcFactory {
           }
         }
 
+        try {
         if (allSatisfyPreCondition) {
+
           // statistics object that is combination of statistics from all relations involved in JOIN
           Statistics stats = new Statistics();
           long prodRows = 1;
@@ -744,7 +778,6 @@ public class StatsRulesProcFactory {
 
             // since new statistics is derived from all relations involved in JOIN,
             // we need to update the state information accordingly
-            stats.updateBasicStatsState(parentStats.getBasicStatsState());
             stats.updateColumnStatsState(parentStats.getColumnStatsState());
           }
 
@@ -812,6 +845,28 @@ public class StatsRulesProcFactory {
           stats.setNumRows(newRowCount);
           stats.setDataSize(StatsUtils.getDataSizeFromColumnStats(newRowCount, outColStats));
           jop.setStatistics(stats);
+          } else {
+
+            // worst case, when no column statistics are available
+            if (parents.size() > 1) {
+              Statistics wcStats = new Statistics();
+              Statistics stp1 = parents.get(0).getStatistics();
+              long numRows = stp1.getNumRows();
+              long avgRowSize = stp1.getAvgRowSize();
+              for (int i = 1; i < parents.size(); i++) {
+                stp1 = parents.get(i).getStatistics();
+                numRows = (numRows * stp1.getNumRows()) / 2;
+                avgRowSize += stp1.getAvgRowSize();
+              }
+              wcStats.setNumRows(numRows);
+              wcStats.setDataSize(numRows * avgRowSize);
+              jop.setStatistics(wcStats);
+            } else {
+              jop.setStatistics(parents.get(0).getStatistics().clone());
+            }
+          }
+        } catch (CloneNotSupportedException e) {
+          throw new SemanticException(ErrorMsg.STATISTICS_CLONING_FAILED.getMsg());
         }
       }
       return null;
@@ -819,6 +874,15 @@ public class StatsRulesProcFactory {
 
     private long getDenominator(List<Long> distinctVals) {
 
+      if(distinctVals.isEmpty()) {
+
+        // TODO: in union20.q the tab alias is not properly propagated down the operator
+        // tree. This happens when UNION ALL is used as sub query. Hence, even if column
+        // statistics are available, the tab alias will be null which will fail to get
+        // proper column statistics. For now assume, worst case in which denominator is 2.
+        return 2;
+      }
+
       // simple join from 2 relations
       // denom = max(v1, v2)
       if (distinctVals.size() <= 2) {
@@ -857,22 +921,31 @@ public class StatsRulesProcFactory {
       Statistics parentStats = parent.getStatistics();
 
       try {
+        long limit = -1;
+        limit = lop.getConf().getLimit();
+
         if (satisfyPrecondition(parentStats)) {
           Statistics stats = parentStats.clone();
-          long limit = -1;
-          limit = lop.getConf().getLimit();
-          if (limit == -1) {
-            limit = lop.getConf().getLeastRows();
-          }
 
-          // if limit is greate than available rows then do not update statistics
+          // if limit is greater than available rows then do not update statistics
           if (limit <= parentStats.getNumRows()) {
             updateStats(stats, limit);
           }
           lop.setStatistics(stats);
         } else {
           if (parentStats != null) {
-            lop.setStatistics(parentStats.clone());
+
+            // in the absence of column statistics, compute data size based on based
+            // on average row size
+            Statistics wcStats = parentStats.clone();
+            if (limit <= parentStats.getNumRows()) {
+              long numRows = limit;
+              long avgRowSize = parentStats.getAvgRowSize();
+              long dataSize = avgRowSize * limit;
+              wcStats.setNumRows(numRows);
+              wcStats.setDataSize(dataSize);
+            }
+            lop.setStatistics(wcStats);
           }
         }
       } catch (CloneNotSupportedException e) {
@@ -909,7 +982,6 @@ public class StatsRulesProcFactory {
                   Statistics parentStats = parent.getStatistics();
                   stats.addToNumRows(parentStats.getNumRows());
                   stats.addToDataSize(parentStats.getDataSize());
-                  stats.updateBasicStatsState(parentStats.getBasicStatsState());
                   stats.updateColumnStatsState(parentStats.getColumnStatsState());
                   stats.addToColumnStats(parentStats.getColumnStats());
                   op.getConf().setStatistics(stats);
@@ -1001,4 +1073,17 @@ public class StatsRulesProcFactory {
     return stats != null && stats.getBasicStatsState().equals(Statistics.State.COMPLETE)
         && !stats.getColumnStatsState().equals(Statistics.State.NONE);
   }
+
+  static Statistics getWorstCaseStats(Statistics stats) throws CloneNotSupportedException {
+    Statistics wcClone = stats.clone();
+    long numRows = wcClone.getNumRows() / 2;
+    long dataSize = wcClone.getDataSize() / 2;
+    long avgRowSize = wcClone.getAvgRowSize();
+    if (numRows > 0) {
+      dataSize = avgRowSize * numRows;
+    }
+    wcClone.setNumRows(numRows);
+    wcClone.setDataSize(dataSize);
+    return wcClone;
+  }
 }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/Statistics.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/Statistics.java?rev=1545372&r1=1545371&r2=1545372&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/Statistics.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/Statistics.java Mon Nov 25 19:15:54 2013
@@ -49,8 +49,8 @@ public class Statistics implements Seria
   }
 
   public Statistics(long nr, long ds) {
-    this.numRows = nr;
-    this.dataSize = ds;
+    this.setNumRows(nr);
+    this.setDataSize(ds);
     this.basicStatsState = State.NONE;
     this.columnStats = null;
     this.columnStatsState = State.NONE;
@@ -62,6 +62,7 @@ public class Statistics implements Seria
 
   public void setNumRows(long numRows) {
     this.numRows = numRows;
+    updateBasicStatsState();
   }
 
   public long getDataSize() {
@@ -70,6 +71,17 @@ public class Statistics implements Seria
 
   public void setDataSize(long dataSize) {
     this.dataSize = dataSize;
+    updateBasicStatsState();
+  }
+
+  private void updateBasicStatsState() {
+    if (numRows <= 0 && dataSize <= 0) {
+      this.basicStatsState = State.NONE;
+    } else if (numRows <= 0 || dataSize <= 0) {
+      this.basicStatsState = State.PARTIAL;
+    } else {
+      this.basicStatsState = State.COMPLETE;
+    }
   }
 
   public State getBasicStatsState() {
@@ -120,10 +132,12 @@ public class Statistics implements Seria
 
   public void addToNumRows(long nr) {
     numRows += nr;
+    updateBasicStatsState();
   }
 
   public void addToDataSize(long rds) {
     dataSize += rds;
+    updateBasicStatsState();
   }
 
   public void setColumnStats(Map<String, ColStatistics> colStats) {
@@ -162,37 +176,14 @@ public class Statistics implements Seria
     }
   }
 
-  // newState
+  //                  newState
   //                  -----------------------------------------
-  // basicStatsState  | COMPLETE          PARTIAL      NONE    |
+  // columnStatsState | COMPLETE          PARTIAL      NONE    |
   //                  |________________________________________|
   //         COMPLETE | COMPLETE          PARTIAL      PARTIAL |
   //          PARTIAL | PARTIAL           PARTIAL      PARTIAL |
   //             NONE | COMPLETE          PARTIAL      NONE    |
   //                  -----------------------------------------
-  public void updateBasicStatsState(State newState) {
-    if (newState.equals(State.PARTIAL)) {
-      basicStatsState = State.PARTIAL;
-    }
-
-    if (newState.equals(State.NONE)) {
-      if (basicStatsState.equals(State.NONE)) {
-        basicStatsState = State.NONE;
-      } else {
-        basicStatsState = State.PARTIAL;
-      }
-    }
-
-    if (newState.equals(State.COMPLETE)) {
-      if (basicStatsState.equals(State.PARTIAL)) {
-        basicStatsState = State.PARTIAL;
-      } else {
-        basicStatsState = State.COMPLETE;
-      }
-    }
-  }
-
-  // similar to the table above for basic stats
   public void updateColumnStatsState(State newState) {
     if (newState.equals(State.PARTIAL)) {
       columnStatsState = State.PARTIAL;
@@ -216,11 +207,11 @@ public class Statistics implements Seria
   }
 
   public long getAvgRowSize() {
-    if (basicStatsState.equals(State.COMPLETE) && numRows != 0) {
+    if (numRows != 0) {
       return dataSize / numRows;
     }
 
-    return 0;
+    return dataSize;
   }
 
   public ColStatistics getColumnStatisticsFromFQColName(String fqColName) {

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java?rev=1545372&r1=1545371&r2=1545372&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java Mon Nov 25 19:15:54 2013
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.plan.Ex
 import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeNullDesc;
 import org.apache.hadoop.hive.ql.plan.Statistics;
+import org.apache.hadoop.hive.ql.plan.Statistics.State;
 import org.apache.hadoop.hive.ql.util.JavaDataModel;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector;
@@ -98,22 +99,6 @@ public class StatsUtils {
         }
       }
 
-      // if basic stats are not available then return
-      if (nr <= 0 && rds <= 0) {
-        stats.setBasicStatsState(Statistics.State.NONE);
-        return stats;
-      }
-
-      // if any basic stats is missing, mark it as partial stats
-      if (nr <= 0 || rds <= 0) {
-        stats.setBasicStatsState(Statistics.State.PARTIAL);
-      }
-
-      // if both are available then we have complete basic stats
-      if (nr > 0 && rds > 0) {
-        stats.setBasicStatsState(Statistics.State.COMPLETE);
-      }
-
       // number of rows -1 means that statistics from metastore is not reliable
       if (nr <= 0) {
         nr = 0;
@@ -177,19 +162,6 @@ public class StatsUtils {
           rds = getSumIgnoreNegatives(dataSizes);
         }
 
-        // basic stats
-        if (nr <= 0 && rds <= 0) {
-          stats.updateBasicStatsState(Statistics.State.NONE);
-        } else if (nr <= 0 || rds <= 0) {
-          stats.updateBasicStatsState(Statistics.State.PARTIAL);
-        } else {
-          if (containsNonPositives(rowCounts) || containsNonPositives(dataSizes)) {
-            stats.updateBasicStatsState(Statistics.State.PARTIAL);
-          } else {
-            stats.updateBasicStatsState(Statistics.State.COMPLETE);
-          }
-        }
-
         // number of rows -1 means that statistics from metastore is not reliable
         if (nr <= 0) {
           nr = 0;
@@ -197,6 +169,11 @@ public class StatsUtils {
         stats.addToNumRows(nr);
         stats.addToDataSize(rds);
 
+        // if atleast a partition does not contain row count then mark basic stats state as PARTIAL
+        if (containsNonPositives(rowCounts)) {
+          stats.setBasicStatsState(State.PARTIAL);
+        }
+
         // column stats
         for (Partition part : partList.getNotDeniedPartns()) {
           List<ColStatistics> colStats = getPartitionColumnStats(table, part, schema, neededColumns);
@@ -219,7 +196,6 @@ public class StatsUtils {
     }
 
     return stats;
-
   }
 
   /**

Modified: hive/trunk/ql/src/test/queries/clientpositive/annotate_stats_groupby.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/annotate_stats_groupby.q?rev=1545372&r1=1545371&r2=1545372&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/annotate_stats_groupby.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/annotate_stats_groupby.q Mon Nov 25 19:15:54 2013
@@ -15,6 +15,18 @@ insert overwrite table loc_orc select * 
 -- numRows: 8 rawDataSize: 796
 explain extended select * from loc_orc;
 
+-- partial column stats
+analyze table loc_orc compute statistics for columns state;
+
+-- inner group by: map - numRows: 8 reduce - numRows: 4
+-- outer group by: map - numRows: 4 reduce numRows: 2
+explain extended select a, c, min(b)
+from ( select state as a, locid as b, count(*) as c
+       from loc_orc
+       group by state,locid
+     ) sq1
+group by a,c;
+
 analyze table loc_orc compute statistics for columns state,locid,zip,year;
 
 -- only one distinct value in year column + 1 NULL value

Modified: hive/trunk/ql/src/test/results/clientpositive/alter_partition_coltype.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/alter_partition_coltype.q.out?rev=1545372&r1=1545371&r2=1545372&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/alter_partition_coltype.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/alter_partition_coltype.q.out Mon Nov 25 19:15:54 2013
@@ -494,10 +494,10 @@ STAGE PLANS:
                   expr: ((ts = 3.0) and (dt = 10))
                   type: boolean
               Statistics:
-                  numRows: 75 dataSize: 0 basicStatsState: COMPLETE colStatsState: COMPLETE
+                  numRows: 75 dataSize: 0 basicStatsState: PARTIAL colStatsState: COMPLETE
               Select Operator
                 Statistics:
-                    numRows: 75 dataSize: 0 basicStatsState: COMPLETE colStatsState: COMPLETE
+                    numRows: 75 dataSize: 0 basicStatsState: PARTIAL colStatsState: COMPLETE
                 Group By Operator
                   aggregations:
                         expr: count()

Modified: hive/trunk/ql/src/test/results/clientpositive/annotate_stats_filter.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/annotate_stats_filter.q.out?rev=1545372&r1=1545371&r2=1545372&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/annotate_stats_filter.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/annotate_stats_filter.q.out Mon Nov 25 19:15:54 2013
@@ -121,7 +121,7 @@ STAGE PLANS:
                   expr: (state = 'OH')
                   type: boolean
               Statistics:
-                  numRows: 8 dataSize: 796 basicStatsState: COMPLETE colStatsState: NONE
+                  numRows: 4 dataSize: 396 basicStatsState: COMPLETE colStatsState: NONE
               Select Operator
                 expressions:
                       expr: state
@@ -134,14 +134,14 @@ STAGE PLANS:
                       type: int
                 outputColumnNames: _col0, _col1, _col2, _col3
                 Statistics:
-                    numRows: 8 dataSize: 796 basicStatsState: COMPLETE colStatsState: NONE
+                    numRows: 4 dataSize: 396 basicStatsState: COMPLETE colStatsState: NONE
                 File Output Operator
                   compressed: false
                   GlobalTableId: 0
 #### A masked pattern was here ####
                   NumFilesPerFileSink: 1
                   Statistics:
-                      numRows: 8 dataSize: 796 basicStatsState: COMPLETE colStatsState: NONE
+                      numRows: 4 dataSize: 396 basicStatsState: COMPLETE colStatsState: NONE
 #### A masked pattern was here ####
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
@@ -1257,7 +1257,7 @@ STAGE PLANS:
                   expr: (not true)
                   type: boolean
               Statistics:
-                  numRows: 0 dataSize: 0 basicStatsState: COMPLETE colStatsState: COMPLETE
+                  numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: COMPLETE
               Select Operator
                 expressions:
                       expr: state
@@ -1270,14 +1270,14 @@ STAGE PLANS:
                       type: int
                 outputColumnNames: _col0, _col1, _col2, _col3
                 Statistics:
-                    numRows: 0 dataSize: 0 basicStatsState: COMPLETE colStatsState: COMPLETE
+                    numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: COMPLETE
                 File Output Operator
                   compressed: false
                   GlobalTableId: 0
 #### A masked pattern was here ####
                   NumFilesPerFileSink: 1
                   Statistics:
-                      numRows: 0 dataSize: 0 basicStatsState: COMPLETE colStatsState: COMPLETE
+                      numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: COMPLETE
 #### A masked pattern was here ####
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat

Modified: hive/trunk/ql/src/test/results/clientpositive/annotate_stats_groupby.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/annotate_stats_groupby.q.out?rev=1545372&r1=1545371&r2=1545372&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/annotate_stats_groupby.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/annotate_stats_groupby.q.out Mon Nov 25 19:15:54 2013
@@ -86,6 +86,309 @@ STAGE PLANS:
             ListSink
 
 
+PREHOOK: query: -- partial column stats
+analyze table loc_orc compute statistics for columns state
+PREHOOK: type: QUERY
+PREHOOK: Input: default@loc_orc
+#### A masked pattern was here ####
+POSTHOOK: query: -- partial column stats
+analyze table loc_orc compute statistics for columns state
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@loc_orc
+#### A masked pattern was here ####
+POSTHOOK: Lineage: loc_orc.locid SIMPLE [(loc_staging)loc_staging.FieldSchema(name:locid, type:int, comment:null), ]
+POSTHOOK: Lineage: loc_orc.state SIMPLE [(loc_staging)loc_staging.FieldSchema(name:state, type:string, comment:null), ]
+POSTHOOK: Lineage: loc_orc.year SIMPLE [(loc_staging)loc_staging.FieldSchema(name:year, type:int, comment:null), ]
+POSTHOOK: Lineage: loc_orc.zip SIMPLE [(loc_staging)loc_staging.FieldSchema(name:zip, type:bigint, comment:null), ]
+PREHOOK: query: -- inner group by: map - numRows: 8 reduce - numRows: 4
+-- outer group by: map - numRows: 4 reduce numRows: 2
+explain extended select a, c, min(b)
+from ( select state as a, locid as b, count(*) as c
+       from loc_orc
+       group by state,locid
+     ) sq1
+group by a,c
+PREHOOK: type: QUERY
+POSTHOOK: query: -- inner group by: map - numRows: 8 reduce - numRows: 4
+-- outer group by: map - numRows: 4 reduce numRows: 2
+explain extended select a, c, min(b)
+from ( select state as a, locid as b, count(*) as c
+       from loc_orc
+       group by state,locid
+     ) sq1
+group by a,c
+POSTHOOK: type: QUERY
+POSTHOOK: Lineage: loc_orc.locid SIMPLE [(loc_staging)loc_staging.FieldSchema(name:locid, type:int, comment:null), ]
+POSTHOOK: Lineage: loc_orc.state SIMPLE [(loc_staging)loc_staging.FieldSchema(name:state, type:string, comment:null), ]
+POSTHOOK: Lineage: loc_orc.year SIMPLE [(loc_staging)loc_staging.FieldSchema(name:year, type:int, comment:null), ]
+POSTHOOK: Lineage: loc_orc.zip SIMPLE [(loc_staging)loc_staging.FieldSchema(name:zip, type:bigint, comment:null), ]
+ABSTRACT SYNTAX TREE:
+  (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME loc_orc))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL state) a) (TOK_SELEXPR (TOK_TABLE_OR_COL locid) b) (TOK_SELEXPR (TOK_FUNCTIONSTAR count) c)) (TOK_GROUPBY (TOK_TABLE_OR_COL state) (TOK_TABLE_OR_COL locid)))) sq1)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL a)) (TOK_SELEXPR (TOK_TABLE_OR_COL c)) (TOK_SELEXPR (TOK_FUNCTION min (TOK_TABLE_OR_COL b)))) (TOK_GROUPBY (TOK_TABLE_OR_COL a) (TOK_TABLE_OR_COL c))))
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Alias -> Map Operator Tree:
+        sq1:loc_orc 
+          TableScan
+            alias: loc_orc
+            Statistics:
+                numRows: 8 dataSize: 796 basicStatsState: COMPLETE colStatsState: PARTIAL
+            GatherStats: false
+            Select Operator
+              expressions:
+                    expr: state
+                    type: string
+                    expr: locid
+                    type: int
+              outputColumnNames: state, locid
+              Statistics:
+                  numRows: 8 dataSize: 796 basicStatsState: COMPLETE colStatsState: PARTIAL
+              Group By Operator
+                aggregations:
+                      expr: count()
+                bucketGroup: false
+                keys:
+                      expr: state
+                      type: string
+                      expr: locid
+                      type: int
+                mode: hash
+                outputColumnNames: _col0, _col1, _col2
+                Statistics:
+                    numRows: 8 dataSize: 688 basicStatsState: COMPLETE colStatsState: PARTIAL
+                Reduce Output Operator
+                  key expressions:
+                        expr: _col0
+                        type: string
+                        expr: _col1
+                        type: int
+                  sort order: ++
+                  Map-reduce partition columns:
+                        expr: _col0
+                        type: string
+                        expr: _col1
+                        type: int
+                  Statistics:
+                      numRows: 8 dataSize: 688 basicStatsState: COMPLETE colStatsState: PARTIAL
+                  tag: -1
+                  value expressions:
+                        expr: _col2
+                        type: bigint
+      Path -> Alias:
+#### A masked pattern was here ####
+      Path -> Partition:
+#### A masked pattern was here ####
+          Partition
+            base file name: loc_orc
+            input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+            output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+            properties:
+              COLUMN_STATS_ACCURATE true
+              bucket_count -1
+              columns state,locid,zip,year
+              columns.types string:int:bigint:int
+              field.delim |
+#### A masked pattern was here ####
+              name default.loc_orc
+              numFiles 1
+              numRows 8
+              rawDataSize 796
+              serialization.ddl struct loc_orc { string state, i32 locid, i64 zip, i32 year}
+              serialization.format |
+              serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
+              totalSize 489
+#### A masked pattern was here ####
+            serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+          
+              input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+              output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+              properties:
+                COLUMN_STATS_ACCURATE true
+                bucket_count -1
+                columns state,locid,zip,year
+                columns.types string:int:bigint:int
+                field.delim |
+#### A masked pattern was here ####
+                name default.loc_orc
+                numFiles 1
+                numRows 8
+                rawDataSize 796
+                serialization.ddl struct loc_orc { string state, i32 locid, i64 zip, i32 year}
+                serialization.format |
+                serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
+                totalSize 489
+#### A masked pattern was here ####
+              serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+              name: default.loc_orc
+            name: default.loc_orc
+      Truncated Path -> Alias:
+        /loc_orc [sq1:loc_orc]
+      Needs Tagging: false
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations:
+                expr: count(VALUE._col0)
+          bucketGroup: false
+          keys:
+                expr: KEY._col0
+                type: string
+                expr: KEY._col1
+                type: int
+          mode: mergepartial
+          outputColumnNames: _col0, _col1, _col2
+          Statistics:
+              numRows: 4 dataSize: 344 basicStatsState: COMPLETE colStatsState: PARTIAL
+          Select Operator
+            expressions:
+                  expr: _col0
+                  type: string
+                  expr: _col1
+                  type: int
+                  expr: _col2
+                  type: bigint
+            outputColumnNames: _col0, _col1, _col2
+            Statistics:
+                numRows: 4 dataSize: 344 basicStatsState: COMPLETE colStatsState: PARTIAL
+            Group By Operator
+              aggregations:
+                    expr: min(_col1)
+              bucketGroup: false
+              keys:
+                    expr: _col0
+                    type: string
+                    expr: _col2
+                    type: bigint
+              mode: hash
+              outputColumnNames: _col0, _col1, _col2
+              Statistics:
+                  numRows: 4 dataSize: 376 basicStatsState: COMPLETE colStatsState: PARTIAL
+              File Output Operator
+                compressed: false
+                GlobalTableId: 0
+#### A masked pattern was here ####
+                NumFilesPerFileSink: 1
+                table:
+                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                    properties:
+                      columns _col0,_col1,_col2
+                      columns.types string,bigint,int
+                      escape.delim \
+                      serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+                    serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+                TotalFiles: 1
+                GatherStats: false
+                MultiFileSpray: false
+
+  Stage: Stage-2
+    Map Reduce
+      Alias -> Map Operator Tree:
+#### A masked pattern was here ####
+          TableScan
+            GatherStats: false
+            Reduce Output Operator
+              key expressions:
+                    expr: _col0
+                    type: string
+                    expr: _col1
+                    type: bigint
+              sort order: ++
+              Map-reduce partition columns:
+                    expr: _col0
+                    type: string
+                    expr: _col1
+                    type: bigint
+              Statistics:
+                  numRows: 4 dataSize: 376 basicStatsState: COMPLETE colStatsState: PARTIAL
+              tag: -1
+              value expressions:
+                    expr: _col2
+                    type: int
+      Path -> Alias:
+#### A masked pattern was here ####
+      Path -> Partition:
+#### A masked pattern was here ####
+          Partition
+            base file name: -mr-10002
+            input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+            output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+            properties:
+              columns _col0,_col1,_col2
+              columns.types string,bigint,int
+              escape.delim \
+              serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+            serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+          
+              input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+              output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+              properties:
+                columns _col0,_col1,_col2
+                columns.types string,bigint,int
+                escape.delim \
+                serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+              serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+      Truncated Path -> Alias:
+#### A masked pattern was here ####
+      Needs Tagging: false
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations:
+                expr: min(VALUE._col0)
+          bucketGroup: false
+          keys:
+                expr: KEY._col0
+                type: string
+                expr: KEY._col1
+                type: bigint
+          mode: mergepartial
+          outputColumnNames: _col0, _col1, _col2
+          Statistics:
+              numRows: 2 dataSize: 188 basicStatsState: COMPLETE colStatsState: PARTIAL
+          Select Operator
+            expressions:
+                  expr: _col0
+                  type: string
+                  expr: _col1
+                  type: bigint
+                  expr: _col2
+                  type: int
+            outputColumnNames: _col0, _col1, _col2
+            Statistics:
+                numRows: 2 dataSize: 196 basicStatsState: COMPLETE colStatsState: PARTIAL
+            File Output Operator
+              compressed: false
+              GlobalTableId: 0
+#### A masked pattern was here ####
+              NumFilesPerFileSink: 1
+              Statistics:
+                  numRows: 2 dataSize: 196 basicStatsState: COMPLETE colStatsState: PARTIAL
+#### A masked pattern was here ####
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  properties:
+                    columns _col0,_col1,_col2
+                    columns.types string:bigint:int
+                    escape.delim \
+                    hive.serialization.extend.nesting.levels true
+                    serialization.format 1
+                    serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              TotalFiles: 1
+              GatherStats: false
+              MultiFileSpray: false
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+
+
 PREHOOK: query: analyze table loc_orc compute statistics for columns state,locid,zip,year
 PREHOOK: type: QUERY
 PREHOOK: Input: default@loc_orc

Modified: hive/trunk/ql/src/test/results/clientpositive/annotate_stats_join.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/annotate_stats_join.q.out?rev=1545372&r1=1545371&r2=1545372&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/annotate_stats_join.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/annotate_stats_join.q.out Mon Nov 25 19:15:54 2013
@@ -1456,7 +1456,7 @@ STAGE PLANS:
           handleSkewJoin: false
           outputColumnNames: _col0, _col1, _col4, _col5
           Statistics:
-              numRows: 0 dataSize: 0 basicStatsState: COMPLETE colStatsState: COMPLETE
+              numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: COMPLETE
           Select Operator
             expressions:
                   expr: _col0
@@ -1469,14 +1469,14 @@ STAGE PLANS:
                   type: string
             outputColumnNames: _col0, _col1, _col2, _col3
             Statistics:
-                numRows: 0 dataSize: 0 basicStatsState: COMPLETE colStatsState: COMPLETE
+                numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: COMPLETE
             File Output Operator
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
               NumFilesPerFileSink: 1
               Statistics:
-                  numRows: 0 dataSize: 0 basicStatsState: COMPLETE colStatsState: COMPLETE
+                  numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: COMPLETE
 #### A masked pattern was here ####
               table:
                   input format: org.apache.hadoop.mapred.TextInputFormat
@@ -1761,7 +1761,7 @@ STAGE PLANS:
           handleSkewJoin: false
           outputColumnNames: _col0, _col1, _col4, _col5, _col8, _col9, _col10, _col11
           Statistics:
-              numRows: 0 dataSize: 0 basicStatsState: COMPLETE colStatsState: COMPLETE
+              numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: COMPLETE
           Select Operator
             expressions:
                   expr: _col0
@@ -1782,14 +1782,14 @@ STAGE PLANS:
                   type: int
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
             Statistics:
-                numRows: 0 dataSize: 0 basicStatsState: COMPLETE colStatsState: COMPLETE
+                numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: COMPLETE
             File Output Operator
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
               NumFilesPerFileSink: 1
               Statistics:
-                  numRows: 0 dataSize: 0 basicStatsState: COMPLETE colStatsState: COMPLETE
+                  numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: COMPLETE
 #### A masked pattern was here ####
               table:
                   input format: org.apache.hadoop.mapred.TextInputFormat

Modified: hive/trunk/ql/src/test/results/clientpositive/annotate_stats_limit.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/annotate_stats_limit.q.out?rev=1545372&r1=1545371&r2=1545372&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/annotate_stats_limit.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/annotate_stats_limit.q.out Mon Nov 25 19:15:54 2013
@@ -231,7 +231,7 @@ STAGE PLANS:
                 numRows: 8 dataSize: 796 basicStatsState: COMPLETE colStatsState: COMPLETE
             Limit
               Statistics:
-                  numRows: 0 dataSize: 0 basicStatsState: COMPLETE colStatsState: COMPLETE
+                  numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: COMPLETE
               ListSink
 
 

Modified: hive/trunk/ql/src/test/results/clientpositive/annotate_stats_part.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/annotate_stats_part.q.out?rev=1545372&r1=1545371&r2=1545372&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/annotate_stats_part.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/annotate_stats_part.q.out Mon Nov 25 19:15:54 2013
@@ -1555,7 +1555,7 @@ STAGE PLANS:
                   expr: (year <> 2001)
                   type: boolean
               Statistics:
-                  numRows: 1 dataSize: 325 basicStatsState: COMPLETE colStatsState: NONE
+                  numRows: 0 dataSize: 162 basicStatsState: PARTIAL colStatsState: NONE
               Select Operator
                 expressions:
                       expr: state
@@ -1564,14 +1564,14 @@ STAGE PLANS:
                       type: int
                 outputColumnNames: _col0, _col1
                 Statistics:
-                    numRows: 1 dataSize: 325 basicStatsState: COMPLETE colStatsState: NONE
+                    numRows: 0 dataSize: 162 basicStatsState: PARTIAL colStatsState: NONE
                 File Output Operator
                   compressed: false
                   GlobalTableId: 0
 #### A masked pattern was here ####
                   NumFilesPerFileSink: 1
                   Statistics:
-                      numRows: 1 dataSize: 325 basicStatsState: COMPLETE colStatsState: NONE
+                      numRows: 0 dataSize: 162 basicStatsState: PARTIAL colStatsState: NONE
 #### A masked pattern was here ####
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat

Modified: hive/trunk/ql/src/test/results/clientpositive/annotate_stats_select.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/annotate_stats_select.q.out?rev=1545372&r1=1545371&r2=1545372&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/annotate_stats_select.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/annotate_stats_select.q.out Mon Nov 25 19:15:54 2013
@@ -932,14 +932,14 @@ STAGE PLANS:
                     type: string
               outputColumnNames: _col0
               Statistics:
-                  numRows: 2 dataSize: 0 basicStatsState: COMPLETE colStatsState: COMPLETE
+                  numRows: 2 dataSize: 0 basicStatsState: PARTIAL colStatsState: COMPLETE
               File Output Operator
                 compressed: false
                 GlobalTableId: 0
 #### A masked pattern was here ####
                 NumFilesPerFileSink: 1
                 Statistics:
-                    numRows: 2 dataSize: 0 basicStatsState: COMPLETE colStatsState: COMPLETE
+                    numRows: 2 dataSize: 0 basicStatsState: PARTIAL colStatsState: COMPLETE
 #### A masked pattern was here ####
                 table:
                     input format: org.apache.hadoop.mapred.TextInputFormat
@@ -4555,21 +4555,21 @@ STAGE PLANS:
                   expr: (not bo1)
                   type: boolean
               Statistics:
-                  numRows: 0 dataSize: 0 basicStatsState: COMPLETE colStatsState: COMPLETE
+                  numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: COMPLETE
               Select Operator
                 expressions:
                       expr: bo1
                       type: boolean
                 outputColumnNames: _col0
                 Statistics:
-                    numRows: 0 dataSize: 0 basicStatsState: COMPLETE colStatsState: COMPLETE
+                    numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: COMPLETE
                 File Output Operator
                   compressed: false
                   GlobalTableId: 0
 #### A masked pattern was here ####
                   NumFilesPerFileSink: 1
                   Statistics:
-                      numRows: 0 dataSize: 0 basicStatsState: COMPLETE colStatsState: COMPLETE
+                      numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: COMPLETE
 #### A masked pattern was here ####
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat

Modified: hive/trunk/ql/src/test/results/clientpositive/auto_join_reordering_values.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/auto_join_reordering_values.q.out?rev=1545372&r1=1545371&r2=1545372&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/auto_join_reordering_values.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/auto_join_reordering_values.q.out Mon Nov 25 19:15:54 2013
@@ -209,6 +209,8 @@ STAGE PLANS:
             1 {VALUE._col1}
           handleSkewJoin: false
           outputColumnNames: _col0, _col3, _col4, _col8
+          Statistics:
+              numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: NONE
           File Output Operator
             compressed: false
             GlobalTableId: 0
@@ -241,6 +243,8 @@ STAGE PLANS:
               Map-reduce partition columns:
                     expr: _col0
                     type: int
+              Statistics:
+                  numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: NONE
               tag: 0
               value expressions:
                     expr: _col8
@@ -347,6 +351,8 @@ STAGE PLANS:
             1 {VALUE._col0}
           handleSkewJoin: false
           outputColumnNames: _col1, _col10, _col11, _col14
+          Statistics:
+              numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: NONE
           File Output Operator
             compressed: false
             GlobalTableId: 0
@@ -379,6 +385,8 @@ STAGE PLANS:
               Map-reduce partition columns:
                     expr: _col10
                     type: int
+              Statistics:
+                  numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: NONE
               tag: 0
               value expressions:
                     expr: _col14
@@ -482,6 +490,8 @@ STAGE PLANS:
             1 
           handleSkewJoin: false
           outputColumnNames: _col1, _col7, _col18
+          Statistics:
+              numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: NONE
           File Output Operator
             compressed: false
             GlobalTableId: 0
@@ -514,6 +524,8 @@ STAGE PLANS:
               Map-reduce partition columns:
                     expr: _col18
                     type: int
+              Statistics:
+                  numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: NONE
               tag: 0
               value expressions:
                     expr: _col7
@@ -615,6 +627,8 @@ STAGE PLANS:
             1 
           handleSkewJoin: false
           outputColumnNames: _col1, _col7
+          Statistics:
+              numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: NONE
           Select Operator
             expressions:
                   expr: _col1
@@ -622,12 +636,18 @@ STAGE PLANS:
                   expr: _col7
                   type: int
             outputColumnNames: _col0, _col1
+            Statistics:
+                numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: NONE
             Limit
+              Statistics:
+                  numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: NONE
               File Output Operator
                 compressed: false
                 GlobalTableId: 0
 #### A masked pattern was here ####
                 NumFilesPerFileSink: 1
+                Statistics:
+                    numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: NONE
 #### A masked pattern was here ####
                 table:
                     input format: org.apache.hadoop.mapred.TextInputFormat

Modified: hive/trunk/ql/src/test/results/clientpositive/auto_sortmerge_join_11.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/auto_sortmerge_join_11.q.out?rev=1545372&r1=1545371&r2=1545372&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/auto_sortmerge_join_11.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/auto_sortmerge_join_11.q.out Mon Nov 25 19:15:54 2013
@@ -175,14 +175,20 @@ STAGE PLANS:
                 1 [Column[key]]
               Position of Big Table: 1
               Select Operator
+                Statistics:
+                    numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: NONE
                 Group By Operator
                   aggregations:
                         expr: count()
                   bucketGroup: false
                   mode: hash
                   outputColumnNames: _col0
+                  Statistics:
+                      numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
                   Reduce Output Operator
                     sort order: 
+                    Statistics:
+                        numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
                     tag: -1
                     value expressions:
                           expr: _col0
@@ -332,16 +338,22 @@ STAGE PLANS:
           bucketGroup: false
           mode: mergepartial
           outputColumnNames: _col0
+          Statistics:
+              numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
           Select Operator
             expressions:
                   expr: _col0
                   type: bigint
             outputColumnNames: _col0
+            Statistics:
+                numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
             File Output Operator
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
               NumFilesPerFileSink: 1
+              Statistics:
+                  numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
 #### A masked pattern was here ####
               table:
                   input format: org.apache.hadoop.mapred.TextInputFormat
@@ -362,6 +374,7 @@ STAGE PLANS:
     Fetch Operator
       limit: -1
 
+
 PREHOOK: query: select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key
 PREHOOK: type: QUERY
 PREHOOK: Input: default@bucket_big
@@ -487,14 +500,20 @@ STAGE PLANS:
                 1 [Column[key]]
               Position of Big Table: 1
               Select Operator
+                Statistics:
+                    numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: NONE
                 Group By Operator
                   aggregations:
                         expr: count()
                   bucketGroup: false
                   mode: hash
                   outputColumnNames: _col0
+                  Statistics:
+                      numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
                   Reduce Output Operator
                     sort order: 
+                    Statistics:
+                        numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
                     tag: -1
                     value expressions:
                           expr: _col0
@@ -644,16 +663,22 @@ STAGE PLANS:
           bucketGroup: false
           mode: mergepartial
           outputColumnNames: _col0
+          Statistics:
+              numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
           Select Operator
             expressions:
                   expr: _col0
                   type: bigint
             outputColumnNames: _col0
+            Statistics:
+                numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
             File Output Operator
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
               NumFilesPerFileSink: 1
+              Statistics:
+                  numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
 #### A masked pattern was here ####
               table:
                   input format: org.apache.hadoop.mapred.TextInputFormat
@@ -674,6 +699,7 @@ STAGE PLANS:
     Fetch Operator
       limit: -1
 
+
 PREHOOK: query: select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key
 PREHOOK: type: QUERY
 PREHOOK: Input: default@bucket_big
@@ -798,16 +824,24 @@ STAGE PLANS:
                 0 [Column[key]]
                 1 [Column[key]]
               Position of Big Table: 1
+              Statistics:
+                  numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: NONE
               BucketMapJoin: true
               Select Operator
+                Statistics:
+                    numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: NONE
                 Group By Operator
                   aggregations:
                         expr: count()
                   bucketGroup: false
                   mode: hash
                   outputColumnNames: _col0
+                  Statistics:
+                      numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
                   Reduce Output Operator
                     sort order: 
+                    Statistics:
+                        numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
                     tag: -1
                     value expressions:
                           expr: _col0
@@ -914,16 +948,22 @@ STAGE PLANS:
           bucketGroup: false
           mode: mergepartial
           outputColumnNames: _col0
+          Statistics:
+              numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
           Select Operator
             expressions:
                   expr: _col0
                   type: bigint
             outputColumnNames: _col0
+            Statistics:
+                numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
             File Output Operator
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
               NumFilesPerFileSink: 1
+              Statistics:
+                  numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
 #### A masked pattern was here ####
               table:
                   input format: org.apache.hadoop.mapred.TextInputFormat

Modified: hive/trunk/ql/src/test/results/clientpositive/auto_sortmerge_join_12.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/auto_sortmerge_join_12.q.out?rev=1545372&r1=1545371&r2=1545372&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/auto_sortmerge_join_12.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/auto_sortmerge_join_12.q.out Mon Nov 25 19:15:54 2013
@@ -346,14 +346,20 @@ STAGE PLANS:
                   1 []
                 Position of Big Table: 0
                 Select Operator
+                  Statistics:
+                      numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: NONE
                   Group By Operator
                     aggregations:
                           expr: count()
                     bucketGroup: false
                     mode: hash
                     outputColumnNames: _col0
+                    Statistics:
+                        numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
                     Reduce Output Operator
                       sort order: 
+                      Statistics:
+                          numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
                       tag: -1
                       value expressions:
                             expr: _col0
@@ -550,16 +556,22 @@ STAGE PLANS:
           bucketGroup: false
           mode: mergepartial
           outputColumnNames: _col0
+          Statistics:
+              numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
           Select Operator
             expressions:
                   expr: _col0
                   type: bigint
             outputColumnNames: _col0
+            Statistics:
+                numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
             File Output Operator
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
               NumFilesPerFileSink: 1
+              Statistics:
+                  numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
 #### A masked pattern was here ####
               table:
                   input format: org.apache.hadoop.mapred.TextInputFormat

Modified: hive/trunk/ql/src/test/results/clientpositive/bucket_map_join_1.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/bucket_map_join_1.q.out?rev=1545372&r1=1545371&r2=1545372&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/bucket_map_join_1.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/bucket_map_join_1.q.out Mon Nov 25 19:15:54 2013
@@ -98,15 +98,23 @@ STAGE PLANS:
                 0 [Column[key], Column[value]]
                 1 [Column[key], Column[value]]
               Position of Big Table: 0
+              Statistics:
+                  numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: NONE
               Select Operator
+                Statistics:
+                    numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: NONE
                 Group By Operator
                   aggregations:
                         expr: count()
                   bucketGroup: false
                   mode: hash
                   outputColumnNames: _col0
+                  Statistics:
+                      numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
                   Reduce Output Operator
                     sort order: 
+                    Statistics:
+                        numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
                     tag: -1
                     value expressions:
                           expr: _col0
@@ -172,16 +180,22 @@ STAGE PLANS:
           bucketGroup: false
           mode: mergepartial
           outputColumnNames: _col0
+          Statistics:
+              numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
           Select Operator
             expressions:
                   expr: _col0
                   type: bigint
             outputColumnNames: _col0
+            Statistics:
+                numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
             File Output Operator
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
               NumFilesPerFileSink: 1
+              Statistics:
+                  numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
 #### A masked pattern was here ####
               table:
                   input format: org.apache.hadoop.mapred.TextInputFormat

Modified: hive/trunk/ql/src/test/results/clientpositive/bucket_map_join_2.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/bucket_map_join_2.q.out?rev=1545372&r1=1545371&r2=1545372&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/bucket_map_join_2.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/bucket_map_join_2.q.out Mon Nov 25 19:15:54 2013
@@ -98,15 +98,23 @@ STAGE PLANS:
                 0 [Column[key], Column[value]]
                 1 [Column[key], Column[value]]
               Position of Big Table: 0
+              Statistics:
+                  numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: NONE
               Select Operator
+                Statistics:
+                    numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: NONE
                 Group By Operator
                   aggregations:
                         expr: count()
                   bucketGroup: false
                   mode: hash
                   outputColumnNames: _col0
+                  Statistics:
+                      numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
                   Reduce Output Operator
                     sort order: 
+                    Statistics:
+                        numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
                     tag: -1
                     value expressions:
                           expr: _col0
@@ -172,16 +180,22 @@ STAGE PLANS:
           bucketGroup: false
           mode: mergepartial
           outputColumnNames: _col0
+          Statistics:
+              numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
           Select Operator
             expressions:
                   expr: _col0
                   type: bigint
             outputColumnNames: _col0
+            Statistics:
+                numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
             File Output Operator
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
               NumFilesPerFileSink: 1
+              Statistics:
+                  numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
 #### A masked pattern was here ####
               table:
                   input format: org.apache.hadoop.mapred.TextInputFormat

Modified: hive/trunk/ql/src/test/results/clientpositive/bucketcontext_1.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/bucketcontext_1.q.out?rev=1545372&r1=1545371&r2=1545372&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/bucketcontext_1.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/bucketcontext_1.q.out Mon Nov 25 19:15:54 2013
@@ -179,16 +179,24 @@ STAGE PLANS:
                 0 [Column[key]]
                 1 [Column[key]]
               Position of Big Table: 1
+              Statistics:
+                  numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: NONE
               BucketMapJoin: true
               Select Operator
+                Statistics:
+                    numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: NONE
                 Group By Operator
                   aggregations:
                         expr: count()
                   bucketGroup: false
                   mode: hash
                   outputColumnNames: _col0
+                  Statistics:
+                      numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
                   Reduce Output Operator
                     sort order: 
+                    Statistics:
+                        numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
                     tag: -1
                     value expressions:
                           expr: _col0
@@ -297,16 +305,22 @@ STAGE PLANS:
           bucketGroup: false
           mode: mergepartial
           outputColumnNames: _col0
+          Statistics:
+              numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
           Select Operator
             expressions:
                   expr: _col0
                   type: bigint
             outputColumnNames: _col0
+            Statistics:
+                numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
             File Output Operator
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
               NumFilesPerFileSink: 1
+              Statistics:
+                  numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
 #### A masked pattern was here ####
               table:
                   input format: org.apache.hadoop.mapred.TextInputFormat

Modified: hive/trunk/ql/src/test/results/clientpositive/bucketcontext_2.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/bucketcontext_2.q.out?rev=1545372&r1=1545371&r2=1545372&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/bucketcontext_2.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/bucketcontext_2.q.out Mon Nov 25 19:15:54 2013
@@ -167,16 +167,24 @@ STAGE PLANS:
                 0 [Column[key]]
                 1 [Column[key]]
               Position of Big Table: 1
+              Statistics:
+                  numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: NONE
               BucketMapJoin: true
               Select Operator
+                Statistics:
+                    numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: NONE
                 Group By Operator
                   aggregations:
                         expr: count()
                   bucketGroup: false
                   mode: hash
                   outputColumnNames: _col0
+                  Statistics:
+                      numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
                   Reduce Output Operator
                     sort order: 
+                    Statistics:
+                        numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
                     tag: -1
                     value expressions:
                           expr: _col0
@@ -285,16 +293,22 @@ STAGE PLANS:
           bucketGroup: false
           mode: mergepartial
           outputColumnNames: _col0
+          Statistics:
+              numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
           Select Operator
             expressions:
                   expr: _col0
                   type: bigint
             outputColumnNames: _col0
+            Statistics:
+                numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
             File Output Operator
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
               NumFilesPerFileSink: 1
+              Statistics:
+                  numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
 #### A masked pattern was here ####
               table:
                   input format: org.apache.hadoop.mapred.TextInputFormat

Modified: hive/trunk/ql/src/test/results/clientpositive/bucketcontext_3.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/bucketcontext_3.q.out?rev=1545372&r1=1545371&r2=1545372&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/bucketcontext_3.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/bucketcontext_3.q.out Mon Nov 25 19:15:54 2013
@@ -209,16 +209,24 @@ STAGE PLANS:
                 0 [Column[key]]
                 1 [Column[key]]
               Position of Big Table: 1
+              Statistics:
+                  numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: NONE
               BucketMapJoin: true
               Select Operator
+                Statistics:
+                    numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: NONE
                 Group By Operator
                   aggregations:
                         expr: count()
                   bucketGroup: false
                   mode: hash
                   outputColumnNames: _col0
+                  Statistics:
+                      numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
                   Reduce Output Operator
                     sort order: 
+                    Statistics:
+                        numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
                     tag: -1
                     value expressions:
                           expr: _col0
@@ -282,16 +290,22 @@ STAGE PLANS:
           bucketGroup: false
           mode: mergepartial
           outputColumnNames: _col0
+          Statistics:
+              numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
           Select Operator
             expressions:
                   expr: _col0
                   type: bigint
             outputColumnNames: _col0
+            Statistics:
+                numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
             File Output Operator
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
               NumFilesPerFileSink: 1
+              Statistics:
+                  numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
 #### A masked pattern was here ####
               table:
                   input format: org.apache.hadoop.mapred.TextInputFormat

Modified: hive/trunk/ql/src/test/results/clientpositive/bucketcontext_4.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/bucketcontext_4.q.out?rev=1545372&r1=1545371&r2=1545372&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/bucketcontext_4.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/bucketcontext_4.q.out Mon Nov 25 19:15:54 2013
@@ -221,16 +221,24 @@ STAGE PLANS:
                 0 [Column[key]]
                 1 [Column[key]]
               Position of Big Table: 1
+              Statistics:
+                  numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: NONE
               BucketMapJoin: true
               Select Operator
+                Statistics:
+                    numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: NONE
                 Group By Operator
                   aggregations:
                         expr: count()
                   bucketGroup: false
                   mode: hash
                   outputColumnNames: _col0
+                  Statistics:
+                      numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
                   Reduce Output Operator
                     sort order: 
+                    Statistics:
+                        numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
                     tag: -1
                     value expressions:
                           expr: _col0
@@ -294,16 +302,22 @@ STAGE PLANS:
           bucketGroup: false
           mode: mergepartial
           outputColumnNames: _col0
+          Statistics:
+              numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
           Select Operator
             expressions:
                   expr: _col0
                   type: bigint
             outputColumnNames: _col0
+            Statistics:
+                numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
             File Output Operator
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
               NumFilesPerFileSink: 1
+              Statistics:
+                  numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
 #### A masked pattern was here ####
               table:
                   input format: org.apache.hadoop.mapred.TextInputFormat

Modified: hive/trunk/ql/src/test/results/clientpositive/bucketcontext_5.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/bucketcontext_5.q.out?rev=1545372&r1=1545371&r2=1545372&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/bucketcontext_5.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/bucketcontext_5.q.out Mon Nov 25 19:15:54 2013
@@ -109,16 +109,24 @@ STAGE PLANS:
                 0 [Column[key]]
                 1 [Column[key]]
               Position of Big Table: 1
+              Statistics:
+                  numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: NONE
               BucketMapJoin: true
               Select Operator
+                Statistics:
+                    numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: NONE
                 Group By Operator
                   aggregations:
                         expr: count()
                   bucketGroup: false
                   mode: hash
                   outputColumnNames: _col0
+                  Statistics:
+                      numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
                   Reduce Output Operator
                     sort order: 
+                    Statistics:
+                        numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
                     tag: -1
                     value expressions:
                           expr: _col0
@@ -184,16 +192,22 @@ STAGE PLANS:
           bucketGroup: false
           mode: mergepartial
           outputColumnNames: _col0
+          Statistics:
+              numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
           Select Operator
             expressions:
                   expr: _col0
                   type: bigint
             outputColumnNames: _col0
+            Statistics:
+                numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
             File Output Operator
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
               NumFilesPerFileSink: 1
+              Statistics:
+                  numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
 #### A masked pattern was here ####
               table:
                   input format: org.apache.hadoop.mapred.TextInputFormat

Modified: hive/trunk/ql/src/test/results/clientpositive/bucketcontext_6.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/bucketcontext_6.q.out?rev=1545372&r1=1545371&r2=1545372&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/bucketcontext_6.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/bucketcontext_6.q.out Mon Nov 25 19:15:54 2013
@@ -123,16 +123,24 @@ STAGE PLANS:
                 0 [Column[key]]
                 1 [Column[key]]
               Position of Big Table: 1
+              Statistics:
+                  numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: NONE
               BucketMapJoin: true
               Select Operator
+                Statistics:
+                    numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: NONE
                 Group By Operator
                   aggregations:
                         expr: count()
                   bucketGroup: false
                   mode: hash
                   outputColumnNames: _col0
+                  Statistics:
+                      numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
                   Reduce Output Operator
                     sort order: 
+                    Statistics:
+                        numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
                     tag: -1
                     value expressions:
                           expr: _col0
@@ -241,16 +249,22 @@ STAGE PLANS:
           bucketGroup: false
           mode: mergepartial
           outputColumnNames: _col0
+          Statistics:
+              numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
           Select Operator
             expressions:
                   expr: _col0
                   type: bigint
             outputColumnNames: _col0
+            Statistics:
+                numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
             File Output Operator
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
               NumFilesPerFileSink: 1
+              Statistics:
+                  numRows: 1 dataSize: 8 basicStatsState: COMPLETE colStatsState: NONE
 #### A masked pattern was here ####
               table:
                   input format: org.apache.hadoop.mapred.TextInputFormat