You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by gu...@apache.org on 2014/01/30 01:24:29 UTC

svn commit: r1562653 [10/10] - in /hive/trunk: itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/ metastore/if/ metastore/src/gen/thrift/gen-cpp/ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ metastore/src/ge...

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java?rev=1562653&r1=1562652&r2=1562653&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java Thu Jan 30 00:24:28 2014
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hive.ql.optimizer;
 
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
@@ -29,6 +30,9 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData;
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.ql.exec.ColumnInfo;
 import org.apache.hadoop.hive.ql.exec.Description;
 import org.apache.hadoop.hive.ql.exec.FetchTask;
@@ -69,8 +73,12 @@ import org.apache.hadoop.hive.serde2.obj
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
 import org.apache.hadoop.hive.serde2.objectinspector.StandardStructObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.thrift.TException;
+
+import com.google.common.collect.Lists;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 
+
 /** There is a set of queries which can be answered entirely from statistics stored in metastore.
  * Examples of such queries are count(*), count(a), max(a), min(b) etc. Hive already collects
  * these basic statistics for query planning purposes. These same statistics can be used to
@@ -80,6 +88,9 @@ import org.apache.hadoop.hive.serde2.typ
  * and than change the plan to answer query entirely using statistics stored in metastore.
  */
 public class StatsOptimizer implements Transform {
+  // TODO: [HIVE-6289] while getting stats from metastore, we currently only get one col at
+  //       a time; this could be improved - get all necessary columns in advance, then use local.
+  // TODO: [HIVE-6292] aggregations could be done directly in metastore. Hive over MySQL!
 
   private static final Log Log = LogFactory.getLog(StatsOptimizer.class);
 
@@ -241,16 +252,19 @@ public class StatsOptimizer implements T
                 if (!StatsSetupConst.areStatsUptoDate(tbl.getParameters())) {
                   Log.debug("Stats for table : " + tbl.getTableName() + " are not upto date.");
                   return null;
-                  }
-                  rowCnt = Long.parseLong(tbl.getProperty(StatsSetupConst.ROW_COUNT));
+                }
+                rowCnt = Long.parseLong(tbl.getProperty(StatsSetupConst.ROW_COUNT));
                 if (rowCnt < 1) {
                   Log.debug("Table doesn't have upto date stats " + tbl.getTableName());
                   return null;
                 }
-                ColumnStatisticsData statData = hive.getMSC().getTableColumnStatistics(
-                    tbl.getDbName(),tbl.getTableName(),colName).
-                    getStatsObjIterator().next().getStatsData();
-                Long nullCnt = getNullcountFor(type, statData);
+                List<ColumnStatisticsObj> stats = hive.getMSC().getTableColumnStatistics(
+                    tbl.getDbName(),tbl.getTableName(), Lists.newArrayList(colName));
+                if (stats.isEmpty()) {
+                  Log.debug("No stats for " + tbl.getTableName() + " column " + colName);
+                  return null;
+                }
+                Long nullCnt = getNullcountFor(type, stats.get(0).getStatsData());
                 if (null == nullCnt) {
                   Log.debug("Unsupported type: " + desc.getTypeString() + " encountered in " +
                       "metadata optimizer for column : " + colName);
@@ -259,23 +273,31 @@ public class StatsOptimizer implements T
                   rowCnt -= nullCnt;
                 }
               } else {
-                for (Partition part : pctx.getPrunedPartitions(tsOp.getConf().getAlias(), tsOp).getPartitions()) {
+                Set<Partition> parts = pctx.getPrunedPartitions(
+                  tsOp.getConf().getAlias(), tsOp).getPartitions();
+                for (Partition part : parts) {
                   if (!StatsSetupConst.areStatsUptoDate(part.getParameters())) {
                     Log.debug("Stats for part : " + part.getSpec() + " are not upto date.");
                     return null;
-                    }
-                    Long partRowCnt = Long.parseLong(part.getParameters()
-                      .get(StatsSetupConst.ROW_COUNT));
+                  }
+                  Long partRowCnt = Long.parseLong(part.getParameters()
+                    .get(StatsSetupConst.ROW_COUNT));
                   if (partRowCnt < 1) {
                     Log.debug("Partition doesn't have upto date stats " + part.getSpec());
                     return null;
                   }
                   rowCnt += partRowCnt;
-                  ColumnStatisticsData statData = hive.getMSC().getPartitionColumnStatistics(
-                    tbl.getDbName(), tbl.getTableName(),part.getName(), colName)
-                    .getStatsObjIterator().next().getStatsData();
+                }
+                Collection<List<ColumnStatisticsObj>> result =
+                    verifyAndGetPartStats(hive, tbl, colName, parts);
+                if (result == null) {
+                  return null; // logging inside
+                }
+                for (List<ColumnStatisticsObj> statObj : result) {
+                  ColumnStatisticsData statData = validateSingleColStat(statObj);
+                  if (statData == null) return null;
                   Long nullCnt = getNullcountFor(type, statData);
-                  if(nullCnt == null) {
+                  if (nullCnt == null) {
                     Log.debug("Unsupported type: " + desc.getTypeString() + " encountered in " +
                         "metadata optimizer for column : " + colName);
                     return null;
@@ -295,12 +317,16 @@ public class StatsOptimizer implements T
             StatType type = getType(colDesc.getTypeString());
             if(!tbl.isPartitioned()) {
               if (!StatsSetupConst.areStatsUptoDate(tbl.getParameters())) {
-                  Log.debug("Stats for table : " + tbl.getTableName() + " are not upto date.");
-                  return null;
-                  }
-              ColumnStatisticsData statData = hive.getMSC().getTableColumnStatistics(
-                tbl.getDbName(),tbl.getTableName(),colName).
-                getStatsObjIterator().next().getStatsData();
+                Log.debug("Stats for table : " + tbl.getTableName() + " are not upto date.");
+                return null;
+              }
+              List<ColumnStatisticsObj> stats = hive.getMSC().getTableColumnStatistics(
+                  tbl.getDbName(),tbl.getTableName(), Lists.newArrayList(colName));
+              if (stats.isEmpty()) {
+                Log.debug("No stats for " + tbl.getTableName() + " column " + colName);
+                return null;
+              }
+              ColumnStatisticsData statData = stats.get(0).getStatsData();
               switch (type) {
               case Integeral:
                 oneRow.add(statData.getLongStats().getHighValue());
@@ -319,19 +345,21 @@ public class StatsOptimizer implements T
                 return null;
               }
             } else {
-              Set<Partition> parts = pctx.getPrunedPartitions(tsOp.getConf().getAlias(), tsOp).getPartitions();
-              switch(type) {
+              Set<Partition> parts = pctx.getPrunedPartitions(
+                tsOp.getConf().getAlias(), tsOp).getPartitions();
+              switch (type) {
               case Integeral: {
                 long maxVal = Long.MIN_VALUE;
-                for (Partition part : parts) {
-                  if (!StatsSetupConst.areStatsUptoDate(part.getParameters())) {
-                    Log.debug("Stats for part : " + part.getSpec() + " are not upto date.");
-                    return null;
-                    }
-                	ColumnStatisticsData statData = hive.getMSC().getPartitionColumnStatistics(
-                      tbl.getDbName(),tbl.getTableName(), part.getName(), colName).
-                      getStatsObjIterator().next().getStatsData();
-                  maxVal = Math.max(maxVal,statData.getLongStats().getHighValue());
+                Collection<List<ColumnStatisticsObj>> result =
+                    verifyAndGetPartStats(hive, tbl, colName, parts);
+                if (result == null) {
+                  return null; // logging inside
+                }
+                for (List<ColumnStatisticsObj> statObj : result) {
+                  ColumnStatisticsData statData = validateSingleColStat(statObj);
+                  if (statData == null) return null;
+                  long curVal = statData.getLongStats().getHighValue();
+                  maxVal = Math.max(maxVal, curVal);
                 }
                 oneRow.add(maxVal);
                 ois.add(PrimitiveObjectInspectorFactory.
@@ -340,15 +368,16 @@ public class StatsOptimizer implements T
               }
               case Double: {
                 double maxVal = Double.MIN_VALUE;
-                for (Partition part : parts) {
-                  if (!StatsSetupConst.areStatsUptoDate(part.getParameters())) {
-                    Log.debug("Stats for part : " + part.getSpec() + " are not upto date.");
-                    return null;
-                    }
-                  ColumnStatisticsData statData = hive.getMSC().getPartitionColumnStatistics(
-                    tbl.getDbName(),tbl.getTableName(), part.getName(), colName).
-                    getStatsObjIterator().next().getStatsData();
-                  maxVal = Math.max(maxVal,statData.getDoubleStats().getHighValue());
+                Collection<List<ColumnStatisticsObj>> result =
+                    verifyAndGetPartStats(hive, tbl, colName, parts);
+                if (result == null) {
+                  return null; // logging inside
+                }
+                for (List<ColumnStatisticsObj> statObj : result) {
+                  ColumnStatisticsData statData = validateSingleColStat(statObj);
+                  if (statData == null) return null;
+                  double curVal = statData.getDoubleStats().getHighValue();
+                  maxVal = Math.max(maxVal, curVal);
                 }
                 oneRow.add(maxVal);
                 ois.add(PrimitiveObjectInspectorFactory.
@@ -370,10 +399,10 @@ public class StatsOptimizer implements T
               if (!StatsSetupConst.areStatsUptoDate(tbl.getParameters())) {
                 Log.debug("Stats for table : " + tbl.getTableName() + " are not upto date.");
                 return null;
-                }
+              }
               ColumnStatisticsData statData = hive.getMSC().getTableColumnStatistics(
-                tbl.getDbName(),tbl.getTableName(),colName).
-                getStatsObjIterator().next().getStatsData();
+                tbl.getDbName(), tbl.getTableName(), Lists.newArrayList(colName))
+                .get(0).getStatsData();
               switch (type) {
               case Integeral:
                 oneRow.add(statData.getLongStats().getLowValue());
@@ -395,15 +424,16 @@ public class StatsOptimizer implements T
               switch(type) {
               case Integeral: {
                 long minVal = Long.MAX_VALUE;
-                for (Partition part : parts) {
-                  if (!StatsSetupConst.areStatsUptoDate(part.getParameters())) {
-                    Log.debug("Stats for part : " + part.getSpec() + " are not upto date.");
-                    return null;
-                    }
-                  ColumnStatisticsData statData = hive.getMSC().getPartitionColumnStatistics(
-                    tbl.getDbName(),tbl.getTableName(), part.getName(), colName).
-                    getStatsObjIterator().next().getStatsData();
-                  minVal = Math.min(minVal,statData.getLongStats().getLowValue());
+                Collection<List<ColumnStatisticsObj>> result =
+                    verifyAndGetPartStats(hive, tbl, colName, parts);
+                if (result == null) {
+                  return null; // logging inside
+                }
+                for (List<ColumnStatisticsObj> statObj : result) {
+                  ColumnStatisticsData statData = validateSingleColStat(statObj);
+                  if (statData == null) return null;
+                  long curVal = statData.getLongStats().getLowValue();
+                  minVal = Math.min(minVal, curVal);
                 }
                 oneRow.add(minVal);
                 ois.add(PrimitiveObjectInspectorFactory.
@@ -412,15 +442,16 @@ public class StatsOptimizer implements T
               }
               case Double: {
                 double minVal = Double.MAX_VALUE;
-                for (Partition part : parts) {
-                  if (!StatsSetupConst.areStatsUptoDate(part.getParameters())) {
-                    Log.debug("Stats for part : " + part.getSpec() + " are not upto date.");
-                    return null;
-                    }
-                  ColumnStatisticsData statData = hive.getMSC().getPartitionColumnStatistics(
-                    tbl.getDbName(),tbl.getTableName(), part.getName(), colName).
-                    getStatsObjIterator().next().getStatsData();
-                  minVal = Math.min(minVal,statData.getDoubleStats().getLowValue());
+                Collection<List<ColumnStatisticsObj>> result =
+                    verifyAndGetPartStats(hive, tbl, colName, parts);
+                if (result == null) {
+                  return null; // logging inside
+                }
+                for (List<ColumnStatisticsObj> statObj : result) {
+                  ColumnStatisticsData statData = validateSingleColStat(statObj);
+                  if (statData == null) return null;
+                  double curVal = statData.getDoubleStats().getLowValue();
+                  minVal = Math.min(minVal, curVal);
                 }
                 oneRow.add(minVal);
                 ois.add(PrimitiveObjectInspectorFactory.
@@ -464,10 +495,42 @@ public class StatsOptimizer implements T
       return null;
     }
 
-    private Long getRowCnt (ParseContext pCtx, TableScanOperator tsOp, Table tbl) throws HiveException {
-        Long rowCnt = 0L;
-      if(tbl.isPartitioned()) {
-        for (Partition part : pctx.getPrunedPartitions(tsOp.getConf().getAlias(), tsOp).getPartitions()) {
+    private ColumnStatisticsData validateSingleColStat(List<ColumnStatisticsObj> statObj) {
+      if (statObj.size() > 1) {
+        Log.error("More than one stat for a single column!");
+        return null;
+      } else if (statObj.isEmpty()) {
+        Log.debug("No stats for some partition and column");
+        return null;
+      }
+      return statObj.get(0).getStatsData();
+    }
+
+    private Collection<List<ColumnStatisticsObj>> verifyAndGetPartStats(
+        Hive hive, Table tbl, String colName, Set<Partition> parts) throws TException {
+      List<String> partNames = new ArrayList<String>(parts.size());
+      for (Partition part : parts) {
+        if (!StatsSetupConst.areStatsUptoDate(part.getParameters())) {
+          Log.debug("Stats for part : " + part.getSpec() + " are not upto date.");
+          return null;
+        }
+        partNames.add(part.getName());
+      }
+      Map<String, List<ColumnStatisticsObj>> result = hive.getMSC().getPartitionColumnStatistics(
+          tbl.getDbName(), tbl.getTableName(), partNames, Lists.newArrayList(colName));
+      if (result.size() != parts.size()) {
+        Log.debug("Received " + result.size() + " stats for " + parts.size() + " partitions");
+        return null;
+      }
+      return result.values();
+    }
+
+    private Long getRowCnt(
+        ParseContext pCtx, TableScanOperator tsOp, Table tbl) throws HiveException {
+      Long rowCnt = 0L;
+      if (tbl.isPartitioned()) {
+        for (Partition part : pctx.getPrunedPartitions(
+            tsOp.getConf().getAlias(), tsOp).getPartitions()) {
           long partRowCnt = Long.parseLong(part.getParameters().get(StatsSetupConst.ROW_COUNT));
           if (partRowCnt < 1) {
             Log.debug("Partition doesn't have upto date stats " + part.getSpec());

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseContext.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseContext.java?rev=1562653&r1=1562652&r2=1562653&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseContext.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseContext.java Thu Jan 30 00:24:28 2014
@@ -297,10 +297,6 @@ public class ParseContext {
     return opToPartList;
   }
 
-  public void setOpToPartList(HashMap<TableScanOperator, PrunedPartitionList> opToPartList) {
-    this.opToPartList = opToPartList;
-  }
-
   /**
    * @return the topToTable
    */

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=1562653&r1=1562652&r2=1562653&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 Thu Jan 30 00:24:28 2014
@@ -18,6 +18,9 @@
 
 package org.apache.hadoop.hive.ql.stats;
 
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
@@ -105,18 +108,16 @@ public class StatsUtils {
     // column level statistics are required only for the columns that are needed
     List<ColumnInfo> schema = tableScanOperator.getSchema().getSignature();
     List<String> neededColumns = tableScanOperator.getNeededColumns();
-    String dbName = table.getDbName();
-    String tabName = table.getTableName();
     boolean fetchColStats =
         HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_STATS_FETCH_COLUMN_STATS);
     float deserFactor =
         HiveConf.getFloatVar(conf, HiveConf.ConfVars.HIVE_STATS_DESERIALIZATION_FACTOR);
 
     if (!table.isPartitioned()) {
-      long nr = getNumRows(dbName, tabName);
-      long ds = getRawDataSize(dbName, tabName);
+      long nr = getNumRows(table);
+      long ds = getRawDataSize(table);
       if (ds <= 0) {
-        ds = getTotalSize(dbName, tabName);
+        ds = getTotalSize(table);
 
         // if data size is still 0 then get file size
         if (ds <= 0) {
@@ -145,121 +146,101 @@ public class StatsUtils {
         colStats = getTableColumnStats(table, schema, neededColumns);
       }
 
-      // if column stats available and if atleast one column doesn't have stats
-      // then mark it as partial
-      if (checkIfColStatsAvailable(colStats) && colStats.contains(null)) {
-        stats.setColumnStatsState(Statistics.State.PARTIAL);
-      }
-
-      // if column stats available and if all columns have stats then mark it
-      // as complete
-      if (checkIfColStatsAvailable(colStats) && !colStats.contains(null)) {
-        stats.setColumnStatsState(Statistics.State.COMPLETE);
-      }
-
-      if (!checkIfColStatsAvailable(colStats)) {
-        // if there is column projection and if we do not have stats then mark
-        // it as NONE. Else we will have estimated stats for const/udf columns
-        if (!neededColumns.isEmpty()) {
-          stats.setColumnStatsState(Statistics.State.NONE);
-        } else {
-          stats.setColumnStatsState(Statistics.State.COMPLETE);
-        }
-      }
+      stats.setColumnStatsState(deriveStatType(colStats, neededColumns));
       stats.addToColumnStats(colStats);
-    } else {
-
+    } else if (partList != null) {
       // For partitioned tables, get the size of all the partitions after pruning
       // the partitions that are not required
-      if (partList != null) {
-        List<String> partNames = Lists.newArrayList();
-        for (Partition part : partList.getNotDeniedPartns()) {
-          partNames.add(part.getName());
-        }
+      List<Long> rowCounts = getBasicStatForPartitions(
+          table, partList.getNotDeniedPartns(), StatsSetupConst.ROW_COUNT);
+      List<Long> dataSizes =  getBasicStatForPartitions(
+          table, partList.getNotDeniedPartns(), StatsSetupConst.RAW_DATA_SIZE);
 
-        List<Long> rowCounts =
-            getBasicStatForPartitions(table, partNames, StatsSetupConst.ROW_COUNT);
-        List<Long> dataSizes =
-            getBasicStatForPartitions(table, partNames, StatsSetupConst.RAW_DATA_SIZE);
+      long nr = getSumIgnoreNegatives(rowCounts);
+      long ds = getSumIgnoreNegatives(dataSizes);
+      if (ds <= 0) {
+        dataSizes = getBasicStatForPartitions(
+            table, partList.getNotDeniedPartns(), StatsSetupConst.TOTAL_SIZE);
+        ds = getSumIgnoreNegatives(dataSizes);
 
-        long nr = getSumIgnoreNegatives(rowCounts);
-        long ds = getSumIgnoreNegatives(dataSizes);
+        // if data size still could not be determined, then fall back to filesytem to get file
+        // sizes
         if (ds <= 0) {
-          dataSizes = getBasicStatForPartitions(table, partNames, StatsSetupConst.TOTAL_SIZE);
-          ds = getSumIgnoreNegatives(dataSizes);
-
-          // if data size still could not be determined, then fall back to filesytem to get file
-          // sizes
-          if (ds <= 0) {
-            dataSizes = getFileSizeForPartitions(conf, partList.getNotDeniedPartns());
-          }
-          ds = getSumIgnoreNegatives(dataSizes);
-
-          ds = (long) (ds * deserFactor);
+          dataSizes = getFileSizeForPartitions(conf, partList.getNotDeniedPartns());
         }
+        ds = getSumIgnoreNegatives(dataSizes);
 
-        int avgRowSize = estimateRowSizeFromSchema(conf, schema, neededColumns);
-        if (avgRowSize > 0) {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Estimated average row size: " + avgRowSize);
-          }
-
-          for (int i = 0; i < rowCounts.size(); i++) {
-            long rc = rowCounts.get(i);
-            long s = dataSizes.get(i);
-            if (rc <= 0 && s > 0) {
-              rc = s / avgRowSize;
-              rowCounts.set(i, rc);
-            }
-
-            if (s <= 0 && rc > 0) {
-              s = rc * avgRowSize;
-              dataSizes.set(i, s);
-            }
-          }
-          nr = getSumIgnoreNegatives(rowCounts);
-          ds = getSumIgnoreNegatives(dataSizes);
+        ds = (long) (ds * deserFactor);
+      }
 
-          // number of rows -1 means that statistics from metastore is not reliable
-          if (nr <= 0) {
-            nr = ds / avgRowSize;
-          }
-        }
-        stats.addToNumRows(nr);
-        stats.addToDataSize(ds);
+      int avgRowSize = estimateRowSizeFromSchema(conf, schema, neededColumns);
+      if (avgRowSize > 0) {
+        setUnknownRcDsToAverage(rowCounts, dataSizes, avgRowSize);
+        nr = getSumIgnoreNegatives(rowCounts);
+        ds = getSumIgnoreNegatives(dataSizes);
 
-        // if atleast a partition does not contain row count then mark basic stats state as PARTIAL
-        if (containsNonPositives(rowCounts)) {
-          stats.setBasicStatsState(State.PARTIAL);
+        // number of rows -1 means that statistics from metastore is not reliable
+        if (nr <= 0) {
+          nr = ds / avgRowSize;
         }
+      }
+      stats.addToNumRows(nr);
+      stats.addToDataSize(ds);
 
-        // column stats
+      // if at least a partition does not contain row count then mark basic stats state as PARTIAL
+      if (containsNonPositives(rowCounts)) {
+        stats.setBasicStatsState(State.PARTIAL);
+      }
+      boolean haveFullStats = fetchColStats;
+      if (fetchColStats) {
+        List<String> partNames = new ArrayList<String>(partList.getNotDeniedPartns().size());
         for (Partition part : partList.getNotDeniedPartns()) {
-          List<ColStatistics> colStats = Lists.newArrayList();
-          if (fetchColStats) {
-            colStats = getPartitionColumnStats(table, part, schema, neededColumns);
-          }
-          if (checkIfColStatsAvailable(colStats) && colStats.contains(null)) {
-            stats.updateColumnStatsState(Statistics.State.PARTIAL);
-          } else if (checkIfColStatsAvailable(colStats) && !colStats.contains(null)) {
-            stats.updateColumnStatsState(Statistics.State.COMPLETE);
-          } else {
-            // if there is column projection and if we do not have stats then mark
-            // it as NONE. Else we will have estimated stats for const/udf columns
-            if (!neededColumns.isEmpty()) {
-              stats.updateColumnStatsState(Statistics.State.NONE);
-            } else {
-              stats.updateColumnStatsState(Statistics.State.COMPLETE);
+          partNames.add(part.getName());
+        }
+        Map<String, List<ColStatistics>> partStats =
+            getPartColumnStats(table, schema, partNames, neededColumns);
+        if (partStats != null) {
+          for (String partName : partNames) {
+            List<ColStatistics> partStat = partStats.get(partName);
+            haveFullStats &= (partStat != null);
+            if (partStat != null) {
+              stats.updateColumnStatsState(deriveStatType(partStat, neededColumns));
+              stats.addToColumnStats(partStat);
             }
           }
-          stats.addToColumnStats(colStats);
         }
       }
+      // There are some partitions with no state (or we didn't fetch any state).
+      // Update the stats with empty list to reflect that in the state/initialize structures.
+      if (!haveFullStats) {
+        List<ColStatistics> emptyStats = Lists.<ColStatistics>newArrayList();
+        stats.addToColumnStats(emptyStats);
+        stats.updateColumnStatsState(deriveStatType(emptyStats, neededColumns));
+      }
     }
-
     return stats;
   }
 
+  private static void setUnknownRcDsToAverage(
+      List<Long> rowCounts, List<Long> dataSizes, int avgRowSize) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Estimated average row size: " + avgRowSize);
+    }
+    for (int i = 0; i < rowCounts.size(); i++) {
+      long rc = rowCounts.get(i);
+      long s = dataSizes.get(i);
+      if (rc <= 0 && s > 0) {
+        rc = s / avgRowSize;
+        rowCounts.set(i, rc);
+      }
+
+      if (s <= 0 && rc > 0) {
+        s = rc * avgRowSize;
+        dataSizes.set(i, s);
+      }
+    }
+  }
+
   public static int estimateRowSizeFromSchema(HiveConf conf, List<ColumnInfo> schema,
       List<String> neededColumns) {
     int avgRowSize = 0;
@@ -361,104 +342,22 @@ public class StatsUtils {
     return result;
   }
 
-  /**
-   * Get the partition level columns statistics from metastore for all the needed columns
-   * @param table
-   *          - table object
-   * @param part
-   *          - partition object
-   * @param schema
-   *          - output schema
-   * @param neededColumns
-   *          - list of needed columns
-   * @return column statistics
-   */
-  public static List<ColStatistics> getPartitionColumnStats(Table table, Partition part,
-      List<ColumnInfo> schema, List<String> neededColumns) {
-
-    String dbName = table.getDbName();
-    String tabName = table.getTableName();
-    String partName = part.getName();
-    List<ColStatistics> colStatistics = Lists.newArrayList();
-    for (ColumnInfo col : schema) {
-      if (!col.isHiddenVirtualCol()) {
-        String colName = col.getInternalName();
-        if (neededColumns.contains(colName)) {
-          String tabAlias = col.getTabAlias();
-          ColStatistics cs = getParitionColumnStatsForColumn(dbName, tabName, partName, colName);
-          if (cs != null) {
-            cs.setTableAlias(tabAlias);
-          }
-          colStatistics.add(cs);
-        }
-      }
-    }
-    return colStatistics;
-  }
-
-  /**
-   * Get the partition level columns statistics from metastore for a specific column
-   * @param dbName
-   *          - database name
-   * @param tabName
-   *          - table name
-   * @param partName
-   *          - partition name
-   * @param colName
-   *          - column name
-   * @return column statistics
-   */
-  public static ColStatistics getParitionColumnStatsForColumn(String dbName, String tabName,
-      String partName, String colName) {
-    try {
-      ColumnStatistics colStats =
-          Hive.get().getPartitionColumnStatistics(dbName, tabName, partName, colName);
-      if (colStats != null) {
-        return getColStatistics(colStats.getStatsObj().get(0), tabName, colName);
-      }
-    } catch (HiveException e) {
-      return null;
-    }
-    return null;
-  }
-
-  /**
-   * Will return true if column statistics for atleast one column is available
-   * @param colStats
-   *          - column stats
-   * @return
-   */
-  private static boolean checkIfColStatsAvailable(List<ColStatistics> colStats) {
-    for (ColStatistics cs : colStats) {
-      if (cs != null) {
-        return true;
+  private static Statistics.State deriveStatType(
+      List<ColStatistics> colStats, List<String> neededColumns) {
+    boolean hasStats = false,
+        hasNull = (colStats == null) || (colStats.size() < neededColumns.size());
+    if (colStats != null) {
+      for (ColStatistics cs : colStats) {
+        boolean isNull = cs == null;
+        hasStats |= !isNull;
+        hasNull |= isNull;
+        if (hasNull && hasStats) break;
       }
     }
-    return false;
-  }
-
-  /**
-   * Get table level column stats for specified column
-   * @param dbName
-   *          - database name
-   * @param tableName
-   *          - table name
-   * @param colName
-   *          - column name
-   * @return column stats
-   */
-  public static ColStatistics getTableColumnStatsForColumn(String dbName, String tableName,
-      String colName) {
-    try {
-      ColumnStatistics colStat = Hive.get().getTableColumnStatistics(dbName, tableName, colName);
-      if (colStat != null) {
-        // there will be only one column statistics object
-        return getColStatistics(colStat.getStatsObj().get(0), tableName, colName);
-      }
-    } catch (HiveException e) {
-      return null;
-    }
-    return null;
+    State result = (hasStats
+        ? (hasNull ? Statistics.State.PARTIAL : Statistics.State.COMPLETE)
+        : (neededColumns.isEmpty() ? Statistics.State.COMPLETE : Statistics.State.NONE));
+    return result;
   }
 
   /**
@@ -536,26 +435,85 @@ public class StatsUtils {
    *          - list of needed columns
    * @return column statistics
    */
-  public static List<ColStatistics> getTableColumnStats(Table table, List<ColumnInfo> schema,
-      List<String> neededColumns) {
+  public static List<ColStatistics> getTableColumnStats(
+      Table table, List<ColumnInfo> schema, List<String> neededColumns) {
+    String dbName = table.getDbName();
+    String tabName = table.getTableName();
+    Map<String, String> colToTabAlias = new HashMap<String, String>(schema.size());
+    List<String> neededColsInTable = processNeededColumns(schema, neededColumns, colToTabAlias);
+    List<ColStatistics> stats = null;
+    try {
+      List<ColumnStatisticsObj> colStat = Hive.get().getTableColumnStatistics(
+          dbName, tabName, neededColsInTable);
+      stats = new ArrayList<ColStatistics>(colStat.size());
+      for (ColumnStatisticsObj statObj : colStat) {
+        ColStatistics cs = getColStatistics(statObj, tabName, statObj.getColName());
+        cs.setTableAlias(colToTabAlias.get(cs.getColumnName()));
+        stats.add(cs);
+      }
+    } catch (HiveException e) {
+      LOG.error("Failed to retrieve table statistics: ", e);
+      stats = null;
+    }
+    return stats;
+  }
 
+  /**
+   * Get table level column statistics from metastore for needed columns
+   * @param table
+   *          - table
+   * @param schema
+   *          - output schema
+   * @param neededColumns
+   *          - list of needed columns
+   * @return column statistics
+   */
+  public static Map<String, List<ColStatistics>> getPartColumnStats(Table table,
+      List<ColumnInfo> schema, List<String> partNames, List<String> neededColumns) {
     String dbName = table.getDbName();
     String tabName = table.getTableName();
-    List<ColStatistics> colStatistics = Lists.newArrayList();
-    for (ColumnInfo col : schema) {
-      if (!col.isHiddenVirtualCol()) {
-        String colName = col.getInternalName();
-        if (neededColumns.contains(colName)) {
-          String tabAlias = col.getTabAlias();
-          ColStatistics cs = getTableColumnStatsForColumn(dbName, tabName, colName);
-          if (cs != null) {
-            cs.setTableAlias(tabAlias);
-          }
-          colStatistics.add(cs);
+    Map<String, String> colToTabAlias = new HashMap<String, String>(schema.size());
+    List<String> neededColsInTable = processNeededColumns(schema, neededColumns, colToTabAlias);
+    Map<String, List<ColStatistics>> stats = null;
+    try {
+      Map<String, List<ColumnStatisticsObj>> colStat = Hive.get().getPartitionColumnStatistics(
+          dbName, tabName, partNames, neededColsInTable);
+      stats = new HashMap<String, List<ColStatistics>>(colStat.size());
+      for (Map.Entry<String, List<ColumnStatisticsObj>> entry : colStat.entrySet()) {
+        List<ColStatistics> partStat = new ArrayList<ColStatistics>(entry.getValue().size());
+        for (ColumnStatisticsObj statObj : entry.getValue()) {
+          ColStatistics cs = getColStatistics(statObj, tabName, statObj.getColName());
+          cs.setTableAlias(colToTabAlias.get(cs.getColumnName()));
+          partStat.add(cs);
         }
+        stats.put(entry.getKey(), partStat);
       }
+    } catch (HiveException e) {
+      LOG.error("Failed to retrieve partitions statistics: ", e);
+      stats = null;
     }
-    return colStatistics;
+    return stats;
+  }
+
+  private static List<String> processNeededColumns(List<ColumnInfo> schema,
+      List<String> neededColumns, Map<String, String> colToTabAlias) {
+    for (ColumnInfo col : schema) {
+      if (col.isHiddenVirtualCol()) continue;
+      colToTabAlias.put(col.getInternalName(), col.getTabAlias());
+    }
+    // Remove hidden virtual columns, as well as needed columns that are not
+    // part of the table. TODO: the latter case should not really happen...
+    List<String> neededColsInTable = null;
+    int limit = neededColumns.size();
+    for (int i = 0; i < limit; ++i) {
+      if (colToTabAlias.containsKey(neededColumns.get(i))) continue;
+      if (neededColsInTable == null) {
+        neededColsInTable = Lists.newArrayList(neededColumns);
+      }
+      neededColsInTable.remove(i--);
+      --limit;
+    }
+    return (neededColsInTable == null) ? neededColumns : neededColsInTable;
   }
 
   /**
@@ -1037,38 +995,26 @@ public class StatsUtils {
 
   /**
    * Get number of rows of a give table
-   * @param dbName
-   *          - database name
-   * @param tabName
-   *          - table name
    * @return number of rows
    */
-  public static long getNumRows(String dbName, String tabName) {
-    return getBasicStatForTable(dbName, tabName, StatsSetupConst.ROW_COUNT);
+  public static long getNumRows(Table table) {
+    return getBasicStatForTable(table, StatsSetupConst.ROW_COUNT);
   }
 
   /**
    * Get raw data size of a give table
-   * @param dbName
-   *          - database name
-   * @param tabName
-   *          - table name
    * @return raw data size
    */
-  public static long getRawDataSize(String dbName, String tabName) {
-    return getBasicStatForTable(dbName, tabName, StatsSetupConst.RAW_DATA_SIZE);
+  public static long getRawDataSize(Table table) {
+    return getBasicStatForTable(table, StatsSetupConst.RAW_DATA_SIZE);
   }
 
   /**
    * Get total size of a give table
-   * @param dbName
-   *          - database name
-   * @param tabName
-   *          - table name
    * @return total size
    */
-  public static long getTotalSize(String dbName, String tabName) {
-    return getBasicStatForTable(dbName, tabName, StatsSetupConst.TOTAL_SIZE);
+  public static long getTotalSize(Table table) {
+    return getBasicStatForTable(table, StatsSetupConst.TOTAL_SIZE);
   }
 
   /**
@@ -1081,15 +1027,7 @@ public class StatsUtils {
    *          - type of stats
    * @return value of stats
    */
-  public static long getBasicStatForTable(String dbName, String tabName, String statType) {
-
-    Table table;
-    try {
-      table = Hive.get().getTable(dbName, tabName);
-    } catch (HiveException e) {
-      return 0;
-    }
-
+  public static long getBasicStatForTable(Table table, String statType) {
     Map<String, String> params = table.getParameters();
     long result = 0;
 
@@ -1107,23 +1045,16 @@ public class StatsUtils {
    * Get basic stats of partitions
    * @param table
    *          - table
-   * @param partNames
-   *          - partition names
+   * @param parts
+   *          - partitions
    * @param statType
    *          - type of stats
    * @return value of stats
    */
-  public static List<Long> getBasicStatForPartitions(Table table, List<String> partNames,
+  public static List<Long> getBasicStatForPartitions(Table table, List<Partition> parts,
       String statType) {
 
     List<Long> stats = Lists.newArrayList();
-    List<Partition> parts;
-    try {
-      parts = Hive.get().getPartitionsByNames(table, partNames);
-    } catch (HiveException e1) {
-      return stats;
-    }
-
     for (Partition part : parts) {
       Map<String, String> params = part.getParameters();
       long result = 0;

Modified: hive/trunk/ql/src/test/results/clientpositive/metadataonly1.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/metadataonly1.q.out?rev=1562653&r1=1562652&r2=1562653&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/metadataonly1.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/metadataonly1.q.out Thu Jan 30 00:24:28 2014
@@ -22,7 +22,7 @@ STAGE PLANS:
           TableScan
             alias: test1
             Statistics:
-                numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: NONE
+                numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: COMPLETE
             GatherStats: false
             Select Operator
               expressions:
@@ -30,7 +30,7 @@ STAGE PLANS:
                     type: string
               outputColumnNames: ds
               Statistics:
-                  numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: NONE
+                  numRows: 0 dataSize: 0 basicStatsState: NONE colStatsState: COMPLETE
               Group By Operator
                 aggregations:
                       expr: max(ds)
@@ -38,11 +38,11 @@ STAGE PLANS:
                 mode: hash
                 outputColumnNames: _col0
                 Statistics:
-                    numRows: 1 dataSize: 84 basicStatsState: COMPLETE colStatsState: NONE
+                    numRows: 1 dataSize: 84 basicStatsState: COMPLETE colStatsState: COMPLETE
                 Reduce Output Operator
                   sort order: 
                   Statistics:
-                      numRows: 1 dataSize: 84 basicStatsState: COMPLETE colStatsState: NONE
+                      numRows: 1 dataSize: 84 basicStatsState: COMPLETE colStatsState: COMPLETE
                   tag: -1
                   value expressions:
                         expr: _col0
@@ -56,21 +56,21 @@ STAGE PLANS:
           mode: mergepartial
           outputColumnNames: _col0
           Statistics:
-              numRows: 1 dataSize: 84 basicStatsState: COMPLETE colStatsState: NONE
+              numRows: 1 dataSize: 84 basicStatsState: COMPLETE colStatsState: COMPLETE
           Select Operator
             expressions:
                   expr: _col0
                   type: string
             outputColumnNames: _col0
             Statistics:
-                numRows: 1 dataSize: 84 basicStatsState: COMPLETE colStatsState: NONE
+                numRows: 1 dataSize: 84 basicStatsState: COMPLETE colStatsState: COMPLETE
             File Output Operator
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
               NumFilesPerFileSink: 1
               Statistics:
-                  numRows: 1 dataSize: 84 basicStatsState: COMPLETE colStatsState: NONE
+                  numRows: 1 dataSize: 84 basicStatsState: COMPLETE colStatsState: COMPLETE
 #### A masked pattern was here ####
               table:
                   input format: org.apache.hadoop.mapred.TextInputFormat