diff --git a/common/src/java/org/apache/hadoop/hive/common/StatsSetupConst.java b/common/src/java/org/apache/hadoop/hive/common/StatsSetupConst.java index 6ddb477..7e1206a 100644 --- a/common/src/java/org/apache/hadoop/hive/common/StatsSetupConst.java +++ b/common/src/java/org/apache/hadoop/hive/common/StatsSetupConst.java @@ -17,8 +17,6 @@ */ package org.apache.hadoop.hive.common; -import java.util.ArrayList; -import java.util.List; /** @@ -64,35 +62,23 @@ public static final String RAW_DATA_SIZE = "rawDataSize"; /** - * @return List of all supported statistics + * list of all supported statistics */ - public static List getSupportedStats() { - List supportedStats = new ArrayList(); - supportedStats.add(NUM_FILES); - supportedStats.add(ROW_COUNT); - supportedStats.add(TOTAL_SIZE); - supportedStats.add(RAW_DATA_SIZE); - return supportedStats; - } + public static final String[] supportedStats = new String[] {NUM_FILES,ROW_COUNT,TOTAL_SIZE,RAW_DATA_SIZE}; + /** - * @return List of all statistics that need to be collected during query execution. These are + * list of all statistics that need to be collected during query execution. These are * statistics that inherently require a scan of the data. */ - public static List getStatsToBeCollected() { - List collectableStats = new ArrayList(); - collectableStats.add(ROW_COUNT); - collectableStats.add(RAW_DATA_SIZE); - return collectableStats; - } + public static final String[] statsRequireCompute = new String[] {ROW_COUNT,RAW_DATA_SIZE}; /** - * @return List of statistics that can be collected quickly without requiring a scan of the data. + * List of statistics that can be collected quickly without requiring a scan of the data. */ - public static List getStatsNoCollection() { - List fastStats = new ArrayList(); - fastStats.add(NUM_FILES); - fastStats.add(TOTAL_SIZE); - return fastStats; - } + public static final String[] statsRequireNoCompute = new String[] {NUM_FILES,TOTAL_SIZE}; + + public static final String STATS_GENERATED_VIA_STATS_TASK = "STATS_GENERATED_VIA_STATS_TASK"; + + public static final String COLUMN_STATS_ACCURATE = "COLUMN_STATS_ACCURATE"; } diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java index 8560e84..1a4ed32 100644 --- a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java +++ b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java @@ -163,8 +163,7 @@ static public void recursiveDelete(File f) throws IOException { * @return True if the passed Parameters Map contains values for all "Fast Stats". */ public static boolean containsAllFastStats(Map partParams) { - List fastStats = StatsSetupConst.getStatsNoCollection(); - for (String stat : fastStats) { + for (String stat : StatsSetupConst.statsRequireNoCompute) { if (!partParams.containsKey(stat)) { return false; } @@ -172,11 +171,6 @@ public static boolean containsAllFastStats(Map partParams) { return true; } - public static boolean updateUnpartitionedTableStatsFast(Database db, Table tbl, Warehouse wh) - throws MetaException { - return updateUnpartitionedTableStatsFast(db, tbl, wh, false, false); - } - public static boolean updateUnpartitionedTableStatsFast(Database db, Table tbl, Warehouse wh, boolean madeDir) throws MetaException { return updateUnpartitionedTableStatsFast(db, tbl, wh, madeDir, false); @@ -215,9 +209,14 @@ public static boolean updateUnpartitionedTableStatsFast(Database db, Table tbl, } params.put(StatsSetupConst.TOTAL_SIZE, Long.toString(tableSize)); LOG.info("Updated size of table " + tbl.getTableName() +" to "+ Long.toString(tableSize)); - if (params.containsKey(StatsSetupConst.ROW_COUNT) || - params.containsKey(StatsSetupConst.RAW_DATA_SIZE)) { - // TODO: Add a MetaStore flag indicating accuracy of these stats and update it here. + if(!params.containsKey(StatsSetupConst.STATS_GENERATED_VIA_STATS_TASK)) { + // invalidate stats requiring scan since this is a regular ddl alter case + for (String stat : StatsSetupConst.statsRequireCompute) { + params.put(stat, "-1"); + } + params.put(StatsSetupConst.COLUMN_STATS_ACCURATE, "false"); + } else { + params.put(StatsSetupConst.COLUMN_STATS_ACCURATE, "true"); } } tbl.setParameters(params); @@ -227,7 +226,7 @@ public static boolean updateUnpartitionedTableStatsFast(Database db, Table tbl, } private static boolean doFastStatsExist(Map parameters) { - for (String stat : StatsSetupConst.getStatsNoCollection()) { + for (String stat : StatsSetupConst.statsRequireNoCompute) { if (!parameters.containsKey(stat)) { return false; } @@ -258,7 +257,7 @@ public static boolean requireCalStats(Configuration hiveConf, Partition oldPart, // requires to calculate stats if new and old have different stats if ((oldPart != null) && (oldPart.getParameters() != null)) { - for (String stat : StatsSetupConst.getStatsNoCollection()) { + for (String stat : StatsSetupConst.statsRequireNoCompute) { if (oldPart.getParameters().containsKey(stat)) { Long oldStat = Long.parseLong(oldPart.getParameters().get(stat)); Long newStat = Long.parseLong(newPart.getParameters().get(stat)); @@ -313,11 +312,14 @@ public static boolean updatePartitionStatsFast(Partition part, Warehouse wh, } params.put(StatsSetupConst.TOTAL_SIZE, Long.toString(partSize)); LOG.warn("Updated size to " + Long.toString(partSize)); - if (params.containsKey(StatsSetupConst.ROW_COUNT) || - params.containsKey(StatsSetupConst.RAW_DATA_SIZE)) { - // The accuracy of these "collectable" stats at this point is suspect unless we know that - // StatsTask was just run before this MetaStore call and populated them. - // TODO: Add a MetaStore flag indicating accuracy of these stats and update it here. + if(!params.containsKey(StatsSetupConst.STATS_GENERATED_VIA_STATS_TASK)) { + // invalidate stats requiring scan since this is a regular ddl alter case + for (String stat : StatsSetupConst.statsRequireCompute) { + params.put(stat, "-1"); + } + params.put(StatsSetupConst.COLUMN_STATS_ACCURATE, "false"); + } else { + params.put(StatsSetupConst.COLUMN_STATS_ACCURATE, "true"); } } part.setParameters(params); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java index 497f0cc..858cbef 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java @@ -66,9 +66,6 @@ private Table table; private List> dpPartSpecs; - private static final List collectableStats = StatsSetupConst.getStatsToBeCollected(); - private static final List supportedStats = StatsSetupConst.getSupportedStats(); - public StatsTask() { super(); dpPartSpecs = null; @@ -84,7 +81,7 @@ public StatsTask() { public Statistics() { stats = new HashMap(); - for (String statType : supportedStats) { + for (String statType : StatsSetupConst.supportedStats) { stats.put(statType, new LongWritable(0L)); } } @@ -108,7 +105,7 @@ public void setStat(String statType, long value) { @Override public String toString() { - return org.apache.commons.lang.StringUtils.join(supportedStats, ", "); + return org.apache.commons.lang.StringUtils.join(StatsSetupConst.supportedStats, ", "); } } @@ -207,7 +204,7 @@ private int aggregateStats() { boolean tableStatsExist = this.existStats(parameters); - for (String statType : supportedStats) { + for (String statType : StatsSetupConst.supportedStats) { if (parameters.containsKey(statType)) { tblStats.setStat(statType, Long.parseLong(parameters.get(statType))); } @@ -226,14 +223,14 @@ private int aggregateStats() { // In case of a non-partitioned table, the key for stats temporary store is "rootDir" if (statsAggregator != null) { String aggKey = Utilities.getHashedStatsPrefix(work.getAggKey(), maxPrefixLength); - updateStats(collectableStats, tblStats, statsAggregator, parameters, + updateStats(StatsSetupConst.statsRequireCompute, tblStats, statsAggregator, parameters, aggKey, atomic); statsAggregator.cleanUp(aggKey); } // The collectable stats for the aggregator needs to be cleared. // For eg. if a file is being loaded, the old number of rows are not valid else if (work.isClearAggregatorStats()) { - for (String statType : collectableStats) { + for (String statType : StatsSetupConst.statsRequireCompute) { if (parameters.containsKey(statType)) { tblStats.setStat(statType, 0L); } @@ -242,9 +239,10 @@ else if (work.isClearAggregatorStats()) { // write table stats to metastore parameters = tTable.getParameters(); - for (String statType : collectableStats) { + for (String statType : StatsSetupConst.statsRequireCompute) { parameters.put(statType, Long.toString(tblStats.getStat(statType))); } + parameters.put(StatsSetupConst.STATS_GENERATED_VIA_STATS_TASK, "true"); tTable.setParameters(parameters); String tableFullName = table.getDbName() + "." + table.getTableName(); @@ -269,7 +267,7 @@ else if (work.isClearAggregatorStats()) { } Map currentValues = new HashMap(); - for (String statType : supportedStats) { + for (String statType : StatsSetupConst.supportedStats) { Long val = parameters.containsKey(statType) ? Long.parseLong(parameters.get(statType)) : 0L; currentValues.put(statType, val); @@ -288,11 +286,11 @@ else if (work.isClearAggregatorStats()) { LOG.info("Stats aggregator : " + partitionID); if (statsAggregator != null) { - updateStats(collectableStats, newPartStats, statsAggregator, + updateStats(StatsSetupConst.statsRequireCompute, newPartStats, statsAggregator, parameters, partitionID, atomic); statsAggregator.cleanUp(partitionID); } else { - for (String statType : collectableStats) { + for (String statType : StatsSetupConst.statsRequireCompute) { // The collectable stats for the aggregator needs to be cleared. // For eg. if a file is being loaded, the old number of rows are not valid if (work.isClearAggregatorStats()) { @@ -320,13 +318,14 @@ else if (work.isClearAggregatorStats()) { // // update the metastore // - for (String statType : supportedStats) { + for (String statType : StatsSetupConst.supportedStats) { long statValue = newPartStats.getStat(statType); if (statValue >= 0) { parameters.put(statType, Long.toString(newPartStats.getStat(statType))); } } + parameters.put(StatsSetupConst.STATS_GENERATED_VIA_STATS_TASK, "true"); tPart.setParameters(parameters); String tableFullName = table.getDbName() + "." + table.getTableName(); db.alterPartition(tableFullName, new Partition(table, tPart)); @@ -364,7 +363,7 @@ private boolean existStats(Map parameters) { || parameters.containsKey(StatsSetupConst.NUM_PARTITIONS); } - private void updateStats(List statsList, Statistics stats, + private void updateStats(String[] statsList, Statistics stats, StatsAggregator statsAggregator, Map parameters, String aggKey, boolean atomic) throws HiveException { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java index 2ac059b..0695a2b 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java @@ -1565,8 +1565,8 @@ private static void clearPartitionStats(org.apache.hadoop.hive.metastore.api.Par if (tpartParams == null) { return; } - List statTypes = StatsSetupConst.getSupportedStats(); - for (String statType : statTypes) { + + for (String statType : StatsSetupConst.supportedStats) { tpartParams.remove(statType); } }