HIVE-12411: Remove counter based stats collection mechanism (Pengcheng Xiong, 
reviewed by Ashutosh Chauhan)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/884ff9ca
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/884ff9ca
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/884ff9ca

Branch: refs/heads/master-fixed
Commit: 884ff9caf4dff634a3501ac6f3daa689d1ce751a
Parents: 6e429d8
Author: Pengcheng Xiong <pxi...@apache.org>
Authored: Tue Nov 24 17:09:40 2015 +0800
Committer: Owen O'Malley <omal...@apache.org>
Committed: Tue Nov 24 12:10:09 2015 -0800

----------------------------------------------------------------------
 .../hadoop/hive/common/StatsSetupConst.java     |  13 -
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   8 +-
 data/conf/llap/hive-site.xml                    |   2 +-
 data/conf/spark/standalone/hive-site.xml        |   2 +-
 data/conf/spark/yarn-client/hive-site.xml       |   2 +-
 data/conf/tez/hive-site.xml                     |   4 +-
 .../hive/ql/stats/CounterStatsAggregator.java   |  82 ----
 .../ql/stats/CounterStatsAggregatorSpark.java   |  58 ---
 .../ql/stats/CounterStatsAggregatorTez.java     |  79 ----
 .../hive/ql/stats/CounterStatsPublisher.java    |  66 ---
 .../hadoop/hive/ql/stats/StatsFactory.java      |  11 -
 .../test/queries/clientpositive/index_bitmap3.q |   1 -
 .../queries/clientpositive/index_bitmap_auto.q  |   1 -
 .../test/queries/clientpositive/stats_counter.q |  16 -
 .../clientpositive/stats_counter_partitioned.q  |  45 --
 .../clientpositive/llap/stats_counter.q.out     | 102 ----
 .../llap/stats_counter_partitioned.q.out        | 465 -------------------
 .../clientpositive/spark/stats_counter.q.out    | 102 ----
 .../spark/stats_counter_partitioned.q.out       | 465 -------------------
 .../results/clientpositive/stats_counter.q.out  | 102 ----
 .../stats_counter_partitioned.q.out             | 465 -------------------
 .../clientpositive/tez/metadataonly1.q.out      |  72 +--
 .../clientpositive/tez/optimize_nullscan.q.out  |  90 ++--
 .../clientpositive/tez/stats_counter.q.out      | 102 ----
 .../tez/stats_counter_partitioned.q.out         | 465 -------------------
 25 files changed, 88 insertions(+), 2732 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/884ff9ca/common/src/java/org/apache/hadoop/hive/common/StatsSetupConst.java
----------------------------------------------------------------------
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 0a44bde..2ff76ee 100644
--- a/common/src/java/org/apache/hadoop/hive/common/StatsSetupConst.java
+++ b/common/src/java/org/apache/hadoop/hive/common/StatsSetupConst.java
@@ -31,19 +31,6 @@ import java.util.Map;
 public class StatsSetupConst {
 
   public enum StatDB {
-    counter {
-      @Override
-      public String getPublisher(Configuration conf) {
-        return "org.apache.hadoop.hive.ql.stats.CounterStatsPublisher"; }
-      @Override
-      public String getAggregator(Configuration conf) {
-        if (HiveConf.getVar(conf, 
HiveConf.ConfVars.HIVE_EXECUTION_ENGINE).equals("tez")) {
-          return "org.apache.hadoop.hive.ql.stats.CounterStatsAggregatorTez";
-        } else if (HiveConf.getVar(conf, 
HiveConf.ConfVars.HIVE_EXECUTION_ENGINE).equals("spark")) {
-          return "org.apache.hadoop.hive.ql.stats.CounterStatsAggregatorSpark";
-        }
-        return "org.apache.hadoop.hive.ql.stats.CounterStatsAggregator"; }
-    },
     fs {
       @Override
       public String getPublisher(Configuration conf) {

http://git-wip-us.apache.org/repos/asf/hive/blob/884ff9ca/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java 
b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index f48403b..fffedd9 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -1354,10 +1354,10 @@ public class HiveConf extends Configuration {
     // Statistics
     HIVESTATSAUTOGATHER("hive.stats.autogather", true,
         "A flag to gather statistics automatically during the INSERT OVERWRITE 
command."),
-    HIVESTATSDBCLASS("hive.stats.dbclass", "fs", new PatternSet("counter", 
"custom", "fs"),
+    HIVESTATSDBCLASS("hive.stats.dbclass", "fs", new PatternSet("custom", 
"fs"),
         "The storage that stores temporary Hive statistics. In filesystem 
based statistics collection ('fs'), \n" +
         "each task writes statistics it has collected in a file on the 
filesystem, which will be aggregated \n" +
-        "after the job has finished. Supported values are fs (filesystem), 
counter, and custom as defined in StatsSetupConst.java."), // 
StatsSetupConst.StatDB
+        "after the job has finished. Supported values are fs (filesystem) and 
custom as defined in StatsSetupConst.java."), // StatsSetupConst.StatDB
     HIVE_STATS_DEFAULT_PUBLISHER("hive.stats.default.publisher", "",
         "The Java class (implementing the StatsPublisher interface) that is 
used by default if hive.stats.dbclass is custom type."),
     HIVE_STATS_DEFAULT_AGGREGATOR("hive.stats.default.aggregator", "",
@@ -1398,10 +1398,6 @@ public class HiveConf extends Configuration {
     HIVE_STATS_KEY_PREFIX_MAX_LENGTH("hive.stats.key.prefix.max.length", 150,
         "Determines if when the prefix of the key used for intermediate stats 
collection\n" +
         "exceeds a certain length, a hash of the key is used instead.  If the 
value < 0 then hashing"),
-    
HIVE_STATS_KEY_PREFIX_RESERVE_LENGTH("hive.stats.key.prefix.reserve.length", 24,
-        "Reserved length for postfix of stats key. Currently only meaningful 
for counter type which should\n" +
-        "keep length of full stats key smaller than max length configured by 
hive.stats.key.prefix.max.length.\n" +
-        "For counter type, it should be bigger than the length of LB spec if 
exists."),
     HIVE_STATS_KEY_PREFIX("hive.stats.key.prefix", "", "", true), // internal 
usage only
     // if length of variable length data type cannot be determined this length 
will be used.
     HIVE_STATS_MAX_VARIABLE_LENGTH("hive.stats.max.variable.length", 100,

http://git-wip-us.apache.org/repos/asf/hive/blob/884ff9ca/data/conf/llap/hive-site.xml
----------------------------------------------------------------------
diff --git a/data/conf/llap/hive-site.xml b/data/conf/llap/hive-site.xml
index 9e15eda..4bf034a 100644
--- a/data/conf/llap/hive-site.xml
+++ b/data/conf/llap/hive-site.xml
@@ -219,7 +219,7 @@
 <property>
   <name>hive.stats.dbclass</name>
   <value>fs</value>
-  <description>The default storatge that stores temporary hive statistics. 
Currently, jdbc, hbase and counter type is supported</description>
+  <description>The default storatge that stores temporary hive statistics. 
Currently, fs type is supported</description>
 </property>
 
 <property>

http://git-wip-us.apache.org/repos/asf/hive/blob/884ff9ca/data/conf/spark/standalone/hive-site.xml
----------------------------------------------------------------------
diff --git a/data/conf/spark/standalone/hive-site.xml 
b/data/conf/spark/standalone/hive-site.xml
index 1a45274..459b52e 100644
--- a/data/conf/spark/standalone/hive-site.xml
+++ b/data/conf/spark/standalone/hive-site.xml
@@ -195,7 +195,7 @@
 <property>
   <name>hive.stats.dbclass</name>
   <value>fs</value>
-  <description>The default storatge that stores temporary hive statistics. 
Currently, jdbc, hbase and counter type is supported</description>
+  <description>The default storatge that stores temporary hive statistics. 
Currently, fs type is supported</description>
 </property>
 
 <property>

http://git-wip-us.apache.org/repos/asf/hive/blob/884ff9ca/data/conf/spark/yarn-client/hive-site.xml
----------------------------------------------------------------------
diff --git a/data/conf/spark/yarn-client/hive-site.xml 
b/data/conf/spark/yarn-client/hive-site.xml
index 346c248..bdab294 100644
--- a/data/conf/spark/yarn-client/hive-site.xml
+++ b/data/conf/spark/yarn-client/hive-site.xml
@@ -195,7 +195,7 @@
 <property>
   <name>hive.stats.dbclass</name>
   <value>fs</value>
-  <description>The default storatge that stores temporary hive statistics. 
Currently, jdbc, hbase and counter type is supported</description>
+  <description>The default storatge that stores temporary hive statistics. 
Currently, fs type is supported</description>
 </property>
 
 <property>

http://git-wip-us.apache.org/repos/asf/hive/blob/884ff9ca/data/conf/tez/hive-site.xml
----------------------------------------------------------------------
diff --git a/data/conf/tez/hive-site.xml b/data/conf/tez/hive-site.xml
index 9e8b04f..ef48d82 100644
--- a/data/conf/tez/hive-site.xml
+++ b/data/conf/tez/hive-site.xml
@@ -218,8 +218,8 @@
 
 <property>
   <name>hive.stats.dbclass</name>
-  <value>counter</value>
-  <description>The default storatge that stores temporary hive statistics. 
Currently, jdbc, hbase and counter type is supported</description>
+  <value>fs</value>
+  <description>The default storatge that stores temporary hive statistics. 
Currently, fs type is supported</description>
 </property>
 
 <property>

http://git-wip-us.apache.org/repos/asf/hive/blob/884ff9ca/ql/src/java/org/apache/hadoop/hive/ql/stats/CounterStatsAggregator.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/stats/CounterStatsAggregator.java 
b/ql/src/java/org/apache/hadoop/hive/ql/stats/CounterStatsAggregator.java
deleted file mode 100644
index 9b66024..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/stats/CounterStatsAggregator.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hive.ql.stats;
-
-import java.io.IOException;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.ql.exec.Task;
-import org.apache.hadoop.hive.ql.exec.mr.ExecDriver;
-import org.apache.hadoop.hive.ql.exec.mr.MapRedTask;
-import org.apache.hadoop.mapred.Counters;
-import org.apache.hadoop.mapred.JobClient;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RunningJob;
-
-public class CounterStatsAggregator implements StatsAggregator {
-
-  private static final Logger LOG = 
LoggerFactory.getLogger(CounterStatsAggregator.class.getName());
-
-  private Counters counters;
-  private JobClient jc;
-
-  @Override
-  public boolean connect(StatsCollectionContext scc) {
-    Task<?> sourceTask = scc.getTask();
-    if (sourceTask instanceof MapRedTask) {
-      try {
-        jc = new JobClient(toJobConf(scc.getHiveConf()));
-        RunningJob job = jc.getJob(((MapRedTask)sourceTask).getJobID());
-        if (job != null) {
-          counters = job.getCounters();
-        }
-      } catch (Exception e) {
-        LOG.error("Failed to get Job instance for " + sourceTask.getJobID(),e);
-      }
-    }
-    return counters != null;
-  }
-
-  private JobConf toJobConf(Configuration hconf) {
-    return hconf instanceof JobConf ? (JobConf)hconf : new JobConf(hconf, 
ExecDriver.class);
-  }
-
-  @Override
-  public String aggregateStats(String counterGrpName, String statType) {
-    long value = 0;
-    if (counters != null) {
-      // In case of counters, aggregation is done by JobTracker / MR AM itself
-      // so no need to aggregate, simply return the counter value for 
requested stat.
-      value = counters.getGroup(counterGrpName).getCounter(statType);
-    }
-    return String.valueOf(value);
-  }
-
-  @Override
-  public boolean closeConnection(StatsCollectionContext scc) {
-    try {
-      jc.close();
-    } catch (IOException e) {
-      LOG.error("Error closing job client for stats aggregator.", e);
-    }
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/884ff9ca/ql/src/java/org/apache/hadoop/hive/ql/stats/CounterStatsAggregatorSpark.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/stats/CounterStatsAggregatorSpark.java 
b/ql/src/java/org/apache/hadoop/hive/ql/stats/CounterStatsAggregatorSpark.java
deleted file mode 100644
index 7ac01a7..0000000
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/stats/CounterStatsAggregatorSpark.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hive.ql.stats;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.ql.exec.Task;
-import org.apache.hadoop.hive.ql.exec.spark.SparkTask;
-import org.apache.hive.spark.counter.SparkCounters;
-
-public class CounterStatsAggregatorSpark
-  implements StatsAggregator {
-
-  private static final Logger LOG = 
LoggerFactory.getLogger(CounterStatsAggregatorSpark.class);
-
-  private SparkCounters sparkCounters;
-
-  @SuppressWarnings("rawtypes")
-  @Override
-  public boolean connect(StatsCollectionContext scc) {
-    SparkTask task = (SparkTask) scc.getTask();
-    sparkCounters = task.getSparkCounters();
-    if (sparkCounters == null) {
-      return false;
-    }
-    return true;
-  }
-
-  @Override
-  public String aggregateStats(String keyPrefix, String statType) {
-    long value = sparkCounters.getValue(keyPrefix, statType);
-    String result = String.valueOf(value);
-    LOG.info(
-      String.format("Counter based stats for (%s, %s) are: %s", keyPrefix, 
statType, result));
-    return result;
-  }
-
-  @Override
-  public boolean closeConnection(StatsCollectionContext scc) {
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/884ff9ca/ql/src/java/org/apache/hadoop/hive/ql/stats/CounterStatsAggregatorTez.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/stats/CounterStatsAggregatorTez.java 
b/ql/src/java/org/apache/hadoop/hive/ql/stats/CounterStatsAggregatorTez.java
deleted file mode 100644
index bb51fea..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/stats/CounterStatsAggregatorTez.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hive.ql.stats;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hive.ql.exec.Task;
-import org.apache.hadoop.hive.ql.exec.tez.TezTask;
-import org.apache.tez.common.counters.TezCounters;
-
-/**
- * This class aggregates stats via counters and does so for Tez Tasks.
- * With dbclass=counters this class will compute table/partition statistics
- * using hadoop counters. They will be published using special keys and
- * then retrieved on the client after the insert/ctas statement ran.
- */
-public class CounterStatsAggregatorTez implements StatsAggregator {
-
-  private static final Logger LOG = 
LoggerFactory.getLogger(CounterStatsAggregatorTez.class.getName());
-
-  private TezCounters counters;
-  private final CounterStatsAggregator mrAggregator;
-  private boolean delegate;
-
-  public CounterStatsAggregatorTez() {
-    mrAggregator = new CounterStatsAggregator();
-  }
-
-  @Override
-  public boolean connect(StatsCollectionContext scc) {
-    Task sourceTask = scc.getTask();
-    if (!(sourceTask instanceof TezTask)) {
-      delegate = true;
-      return mrAggregator.connect(scc);
-    }
-    counters = ((TezTask) sourceTask).getTezCounters();
-    return counters != null;
-  }
-
-  @Override
-  public String aggregateStats(String keyPrefix, String statType) {
-    String result;
-
-    if (delegate) {
-      result = mrAggregator.aggregateStats(keyPrefix, statType);
-    } else {
-      long value = 0;
-      for (String groupName : counters.getGroupNames()) {
-        if (groupName.startsWith(keyPrefix)) {
-          value += 
counters.getGroup(groupName).findCounter(statType).getValue();
-        }
-      }
-      result = String.valueOf(value);
-    }
-    LOG.info("Counter based stats for ("+keyPrefix+") are: "+result);
-    return result;
-  }
-
-  @Override
-  public boolean closeConnection(StatsCollectionContext scc) {
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/884ff9ca/ql/src/java/org/apache/hadoop/hive/ql/stats/CounterStatsPublisher.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/stats/CounterStatsPublisher.java 
b/ql/src/java/org/apache/hadoop/hive/ql/stats/CounterStatsPublisher.java
deleted file mode 100644
index ab3d3cf..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/stats/CounterStatsPublisher.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hive.ql.stats;
-
-import java.util.Map;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hive.ql.exec.MapredContext;
-import org.apache.hadoop.mapred.Reporter;
-
-public class CounterStatsPublisher implements StatsPublisher {
-
-  private static final Logger LOG = 
LoggerFactory.getLogger(CounterStatsPublisher.class.getName());
-
-  private Reporter reporter;
-
-  @Override
-  public boolean init(StatsCollectionContext context) {
-    return true;
-  }
-
-  @Override
-  public boolean connect(StatsCollectionContext statsContext) {
-    MapredContext context = MapredContext.get();
-    if (context == null || context.getReporter() == null) {
-      return false;
-    }
-    reporter = context.getReporter();
-    return true;
-  }
-
-  @Override
-  public boolean publishStat(String fileID, Map<String, String> stats) {
-    for (Map.Entry<String, String> entry : stats.entrySet()) {
-      try {
-        reporter.incrCounter(fileID, entry.getKey(), 
Long.valueOf(entry.getValue()));
-      } catch (Exception e) {
-        LOG.error("Failed to increment counter value " + entry.getValue() + " 
for " + entry.getKey()
-          + ": " + e, e);
-        return false;
-      }
-    }
-    return true;
-  }
-  @Override
-  public boolean closeConnection(StatsCollectionContext context) {
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/884ff9ca/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsFactory.java 
b/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsFactory.java
index a53fcc0..9f4ed67 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsFactory.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.util.ReflectionUtils;
 
 import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVESTATSDBCLASS;
 import static 
org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_STATS_KEY_PREFIX_MAX_LENGTH;
-import static 
org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_STATS_KEY_PREFIX_RESERVE_LENGTH;
 
 /**
  * A factory of stats publisher and aggregator implementations of the
@@ -51,16 +50,6 @@ public final class StatsFactory {
       return -1;
     }
     int maxPrefixLength = HiveConf.getIntVar(conf, 
HIVE_STATS_KEY_PREFIX_MAX_LENGTH);
-    if (HiveConf.getVar(conf, 
HIVESTATSDBCLASS).equalsIgnoreCase(StatDB.counter.name())) {
-      // see org.apache.hadoop.mapred.Counter or 
org.apache.hadoop.mapreduce.MRJobConfig
-      int groupNameMax = conf.getInt("mapreduce.job.counters.group.name.max", 
128);
-      maxPrefixLength = maxPrefixLength < 0 ? groupNameMax :
-          Math.min(maxPrefixLength, groupNameMax);
-    }
-    if (maxPrefixLength > 0) {
-      int reserve = HiveConf.getIntVar(conf, 
HIVE_STATS_KEY_PREFIX_RESERVE_LENGTH);
-      return reserve < 0 ? maxPrefixLength : maxPrefixLength - reserve;
-    }
     return maxPrefixLength;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/884ff9ca/ql/src/test/queries/clientpositive/index_bitmap3.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/index_bitmap3.q 
b/ql/src/test/queries/clientpositive/index_bitmap3.q
index ed29af9..89d363c 100644
--- a/ql/src/test/queries/clientpositive/index_bitmap3.q
+++ b/ql/src/test/queries/clientpositive/index_bitmap3.q
@@ -1,4 +1,3 @@
-set hive.stats.dbclass=counter;
 set hive.stats.autogather=true;
 
 -- SORT_QUERY_RESULTS

http://git-wip-us.apache.org/repos/asf/hive/blob/884ff9ca/ql/src/test/queries/clientpositive/index_bitmap_auto.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/index_bitmap_auto.q 
b/ql/src/test/queries/clientpositive/index_bitmap_auto.q
index 2d434d1..2824094 100644
--- a/ql/src/test/queries/clientpositive/index_bitmap_auto.q
+++ b/ql/src/test/queries/clientpositive/index_bitmap_auto.q
@@ -1,4 +1,3 @@
-set hive.stats.dbclass=counter;
 set hive.stats.autogather=true;
 
 -- SORT_QUERY_RESULTS

http://git-wip-us.apache.org/repos/asf/hive/blob/884ff9ca/ql/src/test/queries/clientpositive/stats_counter.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/stats_counter.q 
b/ql/src/test/queries/clientpositive/stats_counter.q
deleted file mode 100644
index 3c1f132..0000000
--- a/ql/src/test/queries/clientpositive/stats_counter.q
+++ /dev/null
@@ -1,16 +0,0 @@
-set hive.stats.dbclass=counter;
-set hive.stats.autogather=false;
-
--- by analyze
-create table dummy1 as select * from src;
-
-analyze table dummy1 compute statistics;
-desc formatted dummy1;
-
-set hive.stats.dbclass=counter;
-set hive.stats.autogather=true;
-
--- by autogather
-create table dummy2 as select * from src;
-
-desc formatted dummy2;

http://git-wip-us.apache.org/repos/asf/hive/blob/884ff9ca/ql/src/test/queries/clientpositive/stats_counter_partitioned.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/stats_counter_partitioned.q 
b/ql/src/test/queries/clientpositive/stats_counter_partitioned.q
deleted file mode 100644
index e1274c0..0000000
--- a/ql/src/test/queries/clientpositive/stats_counter_partitioned.q
+++ /dev/null
@@ -1,45 +0,0 @@
-set hive.stats.dbclass=counter;
-set hive.stats.autogather=true;
-set hive.exec.dynamic.partition.mode=nonstrict;
-
--- partitioned table analyze 
-
-create table dummy (key string, value string) partitioned by (ds string, hr 
string);
-
-load data local inpath '../../data/files/kv1.txt' into table dummy partition 
(ds='2008',hr='12');
-load data local inpath '../../data/files/kv1.txt' into table dummy partition 
(ds='2008',hr='11');
-
-analyze table dummy partition (ds,hr) compute statistics;
-describe formatted dummy partition (ds='2008', hr='11');
-describe formatted dummy partition (ds='2008', hr='12');
-
-drop table dummy;
-
--- static partitioned table on insert
-
-create table dummy (key string, value string) partitioned by (ds string, hr 
string);
-
-insert overwrite table dummy partition (ds='10',hr='11') select * from src;
-insert overwrite table dummy partition (ds='10',hr='12') select * from src;
-
-describe formatted dummy partition (ds='10', hr='11');
-describe formatted dummy partition (ds='10', hr='12');
-
-drop table dummy;
-
--- dynamic partitioned table on insert
-
-create table dummy (key int) partitioned by (hr int);
-                                                                               
                       
-CREATE TABLE tbl(key int, value int) ROW FORMAT DELIMITED FIELDS TERMINATED BY 
'|';
-LOAD DATA LOCAL INPATH '../../data/files/tbl.txt' OVERWRITE INTO TABLE tbl;    
                       
-                                                                               
                       
-insert overwrite table dummy partition (hr) select * from tbl;
-
-describe formatted dummy partition (hr=1997);
-describe formatted dummy partition (hr=1994);
-describe formatted dummy partition (hr=1998);
-describe formatted dummy partition (hr=1996);
-
-drop table tbl;
-drop table dummy; 

http://git-wip-us.apache.org/repos/asf/hive/blob/884ff9ca/ql/src/test/results/clientpositive/llap/stats_counter.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/stats_counter.q.out 
b/ql/src/test/results/clientpositive/llap/stats_counter.q.out
deleted file mode 100644
index 8b3dcea..0000000
--- a/ql/src/test/results/clientpositive/llap/stats_counter.q.out
+++ /dev/null
@@ -1,102 +0,0 @@
-PREHOOK: query: -- by analyze
-create table dummy1 as select * from src
-PREHOOK: type: CREATETABLE_AS_SELECT
-PREHOOK: Input: default@src
-PREHOOK: Output: database:default
-PREHOOK: Output: default@dummy1
-POSTHOOK: query: -- by analyze
-create table dummy1 as select * from src
-POSTHOOK: type: CREATETABLE_AS_SELECT
-POSTHOOK: Input: default@src
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@dummy1
-PREHOOK: query: analyze table dummy1 compute statistics
-PREHOOK: type: QUERY
-PREHOOK: Input: default@dummy1
-PREHOOK: Output: default@dummy1
-POSTHOOK: query: analyze table dummy1 compute statistics
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@dummy1
-POSTHOOK: Output: default@dummy1
-PREHOOK: query: desc formatted dummy1
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@dummy1
-POSTHOOK: query: desc formatted dummy1
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@dummy1
-# col_name             data_type               comment             
-                
-key                    string                                      
-value                  string                                      
-                
-# Detailed Table Information            
-Database:              default                  
-#### A masked pattern was here ####
-Retention:             0                        
-#### A masked pattern was here ####
-Table Type:            MANAGED_TABLE            
-Table Parameters:               
-       COLUMN_STATS_ACCURATE   true                
-       numFiles                1                   
-       numRows                 500                 
-       rawDataSize             5312                
-       totalSize               5812                
-#### A masked pattern was here ####
-                
-# Storage Information           
-SerDe Library:         org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe      
 
-InputFormat:           org.apache.hadoop.mapred.TextInputFormat         
-OutputFormat:          
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat       
-Compressed:            No                       
-Num Buckets:           -1                       
-Bucket Columns:        []                       
-Sort Columns:          []                       
-Storage Desc Params:            
-       serialization.format    1                   
-PREHOOK: query: -- by autogather
-create table dummy2 as select * from src
-PREHOOK: type: CREATETABLE_AS_SELECT
-PREHOOK: Input: default@src
-PREHOOK: Output: database:default
-PREHOOK: Output: default@dummy2
-POSTHOOK: query: -- by autogather
-create table dummy2 as select * from src
-POSTHOOK: type: CREATETABLE_AS_SELECT
-POSTHOOK: Input: default@src
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@dummy2
-PREHOOK: query: desc formatted dummy2
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@dummy2
-POSTHOOK: query: desc formatted dummy2
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@dummy2
-# col_name             data_type               comment             
-                
-key                    string                                      
-value                  string                                      
-                
-# Detailed Table Information            
-Database:              default                  
-#### A masked pattern was here ####
-Retention:             0                        
-#### A masked pattern was here ####
-Table Type:            MANAGED_TABLE            
-Table Parameters:               
-       COLUMN_STATS_ACCURATE   true                
-       numFiles                1                   
-       numRows                 500                 
-       rawDataSize             5312                
-       totalSize               5812                
-#### A masked pattern was here ####
-                
-# Storage Information           
-SerDe Library:         org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe      
 
-InputFormat:           org.apache.hadoop.mapred.TextInputFormat         
-OutputFormat:          
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat       
-Compressed:            No                       
-Num Buckets:           -1                       
-Bucket Columns:        []                       
-Sort Columns:          []                       
-Storage Desc Params:            
-       serialization.format    1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/884ff9ca/ql/src/test/results/clientpositive/llap/stats_counter_partitioned.q.out
----------------------------------------------------------------------
diff --git 
a/ql/src/test/results/clientpositive/llap/stats_counter_partitioned.q.out 
b/ql/src/test/results/clientpositive/llap/stats_counter_partitioned.q.out
deleted file mode 100644
index 626dcff..0000000
--- a/ql/src/test/results/clientpositive/llap/stats_counter_partitioned.q.out
+++ /dev/null
@@ -1,465 +0,0 @@
-PREHOOK: query: -- partitioned table analyze 
-
-create table dummy (key string, value string) partitioned by (ds string, hr 
string)
-PREHOOK: type: CREATETABLE
-PREHOOK: Output: database:default
-PREHOOK: Output: default@dummy
-POSTHOOK: query: -- partitioned table analyze 
-
-create table dummy (key string, value string) partitioned by (ds string, hr 
string)
-POSTHOOK: type: CREATETABLE
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@dummy
-PREHOOK: query: load data local inpath '../../data/files/kv1.txt' into table 
dummy partition (ds='2008',hr='12')
-PREHOOK: type: LOAD
-#### A masked pattern was here ####
-PREHOOK: Output: default@dummy
-POSTHOOK: query: load data local inpath '../../data/files/kv1.txt' into table 
dummy partition (ds='2008',hr='12')
-POSTHOOK: type: LOAD
-#### A masked pattern was here ####
-POSTHOOK: Output: default@dummy
-POSTHOOK: Output: default@dummy@ds=2008/hr=12
-PREHOOK: query: load data local inpath '../../data/files/kv1.txt' into table 
dummy partition (ds='2008',hr='11')
-PREHOOK: type: LOAD
-#### A masked pattern was here ####
-PREHOOK: Output: default@dummy
-POSTHOOK: query: load data local inpath '../../data/files/kv1.txt' into table 
dummy partition (ds='2008',hr='11')
-POSTHOOK: type: LOAD
-#### A masked pattern was here ####
-POSTHOOK: Output: default@dummy
-POSTHOOK: Output: default@dummy@ds=2008/hr=11
-PREHOOK: query: analyze table dummy partition (ds,hr) compute statistics
-PREHOOK: type: QUERY
-PREHOOK: Input: default@dummy
-PREHOOK: Input: default@dummy@ds=2008/hr=11
-PREHOOK: Input: default@dummy@ds=2008/hr=12
-PREHOOK: Output: default@dummy
-PREHOOK: Output: default@dummy@ds=2008/hr=11
-PREHOOK: Output: default@dummy@ds=2008/hr=12
-POSTHOOK: query: analyze table dummy partition (ds,hr) compute statistics
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@dummy
-POSTHOOK: Input: default@dummy@ds=2008/hr=11
-POSTHOOK: Input: default@dummy@ds=2008/hr=12
-POSTHOOK: Output: default@dummy
-POSTHOOK: Output: default@dummy@ds=2008/hr=11
-POSTHOOK: Output: default@dummy@ds=2008/hr=12
-PREHOOK: query: describe formatted dummy partition (ds='2008', hr='11')
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@dummy
-POSTHOOK: query: describe formatted dummy partition (ds='2008', hr='11')
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@dummy
-# col_name             data_type               comment             
-                
-key                    string                                      
-value                  string                                      
-                
-# Partition Information                 
-# col_name             data_type               comment             
-                
-ds                     string                                      
-hr                     string                                      
-                
-# Detailed Partition Information                
-Partition Value:       [2008, 11]               
-Database:              default                  
-Table:                 dummy                    
-#### A masked pattern was here ####
-Partition Parameters:           
-       COLUMN_STATS_ACCURATE   true                
-       numFiles                1                   
-       numRows                 500                 
-       rawDataSize             5312                
-       totalSize               5812                
-#### A masked pattern was here ####
-                
-# Storage Information           
-SerDe Library:         org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe      
 
-InputFormat:           org.apache.hadoop.mapred.TextInputFormat         
-OutputFormat:          
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat       
-Compressed:            No                       
-Num Buckets:           -1                       
-Bucket Columns:        []                       
-Sort Columns:          []                       
-Storage Desc Params:            
-       serialization.format    1                   
-PREHOOK: query: describe formatted dummy partition (ds='2008', hr='12')
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@dummy
-POSTHOOK: query: describe formatted dummy partition (ds='2008', hr='12')
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@dummy
-# col_name             data_type               comment             
-                
-key                    string                                      
-value                  string                                      
-                
-# Partition Information                 
-# col_name             data_type               comment             
-                
-ds                     string                                      
-hr                     string                                      
-                
-# Detailed Partition Information                
-Partition Value:       [2008, 12]               
-Database:              default                  
-Table:                 dummy                    
-#### A masked pattern was here ####
-Partition Parameters:           
-       COLUMN_STATS_ACCURATE   true                
-       numFiles                1                   
-       numRows                 500                 
-       rawDataSize             5312                
-       totalSize               5812                
-#### A masked pattern was here ####
-                
-# Storage Information           
-SerDe Library:         org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe      
 
-InputFormat:           org.apache.hadoop.mapred.TextInputFormat         
-OutputFormat:          
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat       
-Compressed:            No                       
-Num Buckets:           -1                       
-Bucket Columns:        []                       
-Sort Columns:          []                       
-Storage Desc Params:            
-       serialization.format    1                   
-PREHOOK: query: drop table dummy
-PREHOOK: type: DROPTABLE
-PREHOOK: Input: default@dummy
-PREHOOK: Output: default@dummy
-POSTHOOK: query: drop table dummy
-POSTHOOK: type: DROPTABLE
-POSTHOOK: Input: default@dummy
-POSTHOOK: Output: default@dummy
-PREHOOK: query: -- static partitioned table on insert
-
-create table dummy (key string, value string) partitioned by (ds string, hr 
string)
-PREHOOK: type: CREATETABLE
-PREHOOK: Output: database:default
-PREHOOK: Output: default@dummy
-POSTHOOK: query: -- static partitioned table on insert
-
-create table dummy (key string, value string) partitioned by (ds string, hr 
string)
-POSTHOOK: type: CREATETABLE
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@dummy
-PREHOOK: query: insert overwrite table dummy partition (ds='10',hr='11') 
select * from src
-PREHOOK: type: QUERY
-PREHOOK: Input: default@src
-PREHOOK: Output: default@dummy@ds=10/hr=11
-POSTHOOK: query: insert overwrite table dummy partition (ds='10',hr='11') 
select * from src
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@src
-POSTHOOK: Output: default@dummy@ds=10/hr=11
-POSTHOOK: Lineage: dummy PARTITION(ds=10,hr=11).key SIMPLE 
[(src)src.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: dummy PARTITION(ds=10,hr=11).value SIMPLE 
[(src)src.FieldSchema(name:value, type:string, comment:default), ]
-PREHOOK: query: insert overwrite table dummy partition (ds='10',hr='12') 
select * from src
-PREHOOK: type: QUERY
-PREHOOK: Input: default@src
-PREHOOK: Output: default@dummy@ds=10/hr=12
-POSTHOOK: query: insert overwrite table dummy partition (ds='10',hr='12') 
select * from src
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@src
-POSTHOOK: Output: default@dummy@ds=10/hr=12
-POSTHOOK: Lineage: dummy PARTITION(ds=10,hr=12).key SIMPLE 
[(src)src.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: dummy PARTITION(ds=10,hr=12).value SIMPLE 
[(src)src.FieldSchema(name:value, type:string, comment:default), ]
-PREHOOK: query: describe formatted dummy partition (ds='10', hr='11')
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@dummy
-POSTHOOK: query: describe formatted dummy partition (ds='10', hr='11')
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@dummy
-# col_name             data_type               comment             
-                
-key                    string                                      
-value                  string                                      
-                
-# Partition Information                 
-# col_name             data_type               comment             
-                
-ds                     string                                      
-hr                     string                                      
-                
-# Detailed Partition Information                
-Partition Value:       [10, 11]                 
-Database:              default                  
-Table:                 dummy                    
-#### A masked pattern was here ####
-Partition Parameters:           
-       COLUMN_STATS_ACCURATE   true                
-       numFiles                1                   
-       numRows                 500                 
-       rawDataSize             5312                
-       totalSize               5812                
-#### A masked pattern was here ####
-                
-# Storage Information           
-SerDe Library:         org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe      
 
-InputFormat:           org.apache.hadoop.mapred.TextInputFormat         
-OutputFormat:          
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat       
-Compressed:            No                       
-Num Buckets:           -1                       
-Bucket Columns:        []                       
-Sort Columns:          []                       
-Storage Desc Params:            
-       serialization.format    1                   
-PREHOOK: query: describe formatted dummy partition (ds='10', hr='12')
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@dummy
-POSTHOOK: query: describe formatted dummy partition (ds='10', hr='12')
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@dummy
-# col_name             data_type               comment             
-                
-key                    string                                      
-value                  string                                      
-                
-# Partition Information                 
-# col_name             data_type               comment             
-                
-ds                     string                                      
-hr                     string                                      
-                
-# Detailed Partition Information                
-Partition Value:       [10, 12]                 
-Database:              default                  
-Table:                 dummy                    
-#### A masked pattern was here ####
-Partition Parameters:           
-       COLUMN_STATS_ACCURATE   true                
-       numFiles                1                   
-       numRows                 500                 
-       rawDataSize             5312                
-       totalSize               5812                
-#### A masked pattern was here ####
-                
-# Storage Information           
-SerDe Library:         org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe      
 
-InputFormat:           org.apache.hadoop.mapred.TextInputFormat         
-OutputFormat:          
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat       
-Compressed:            No                       
-Num Buckets:           -1                       
-Bucket Columns:        []                       
-Sort Columns:          []                       
-Storage Desc Params:            
-       serialization.format    1                   
-PREHOOK: query: drop table dummy
-PREHOOK: type: DROPTABLE
-PREHOOK: Input: default@dummy
-PREHOOK: Output: default@dummy
-POSTHOOK: query: drop table dummy
-POSTHOOK: type: DROPTABLE
-POSTHOOK: Input: default@dummy
-POSTHOOK: Output: default@dummy
-PREHOOK: query: -- dynamic partitioned table on insert
-
-create table dummy (key int) partitioned by (hr int)
-PREHOOK: type: CREATETABLE
-PREHOOK: Output: database:default
-PREHOOK: Output: default@dummy
-POSTHOOK: query: -- dynamic partitioned table on insert
-
-create table dummy (key int) partitioned by (hr int)
-POSTHOOK: type: CREATETABLE
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@dummy
-PREHOOK: query: CREATE TABLE tbl(key int, value int) ROW FORMAT DELIMITED 
FIELDS TERMINATED BY '|'
-PREHOOK: type: CREATETABLE
-PREHOOK: Output: database:default
-PREHOOK: Output: default@tbl
-POSTHOOK: query: CREATE TABLE tbl(key int, value int) ROW FORMAT DELIMITED 
FIELDS TERMINATED BY '|'
-POSTHOOK: type: CREATETABLE
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@tbl
-PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/tbl.txt' OVERWRITE 
INTO TABLE tbl
-PREHOOK: type: LOAD
-#### A masked pattern was here ####
-PREHOOK: Output: default@tbl
-POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/tbl.txt' OVERWRITE 
INTO TABLE tbl
-POSTHOOK: type: LOAD
-#### A masked pattern was here ####
-POSTHOOK: Output: default@tbl
-PREHOOK: query: insert overwrite table dummy partition (hr) select * from tbl
-PREHOOK: type: QUERY
-PREHOOK: Input: default@tbl
-PREHOOK: Output: default@dummy
-POSTHOOK: query: insert overwrite table dummy partition (hr) select * from tbl
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@tbl
-POSTHOOK: Output: default@dummy@hr=1994
-POSTHOOK: Output: default@dummy@hr=1996
-POSTHOOK: Output: default@dummy@hr=1997
-POSTHOOK: Output: default@dummy@hr=1998
-POSTHOOK: Lineage: dummy PARTITION(hr=1994).key SIMPLE 
[(tbl)tbl.FieldSchema(name:key, type:int, comment:null), ]
-POSTHOOK: Lineage: dummy PARTITION(hr=1996).key SIMPLE 
[(tbl)tbl.FieldSchema(name:key, type:int, comment:null), ]
-POSTHOOK: Lineage: dummy PARTITION(hr=1997).key SIMPLE 
[(tbl)tbl.FieldSchema(name:key, type:int, comment:null), ]
-POSTHOOK: Lineage: dummy PARTITION(hr=1998).key SIMPLE 
[(tbl)tbl.FieldSchema(name:key, type:int, comment:null), ]
-PREHOOK: query: describe formatted dummy partition (hr=1997)
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@dummy
-POSTHOOK: query: describe formatted dummy partition (hr=1997)
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@dummy
-# col_name             data_type               comment             
-                
-key                    int                                         
-                
-# Partition Information                 
-# col_name             data_type               comment             
-                
-hr                     int                                         
-                
-# Detailed Partition Information                
-Partition Value:       [1997]                   
-Database:              default                  
-Table:                 dummy                    
-#### A masked pattern was here ####
-Partition Parameters:           
-       COLUMN_STATS_ACCURATE   true                
-       numFiles                1                   
-       numRows                 6                   
-       rawDataSize             6                   
-       totalSize               12                  
-#### A masked pattern was here ####
-                
-# Storage Information           
-SerDe Library:         org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe      
 
-InputFormat:           org.apache.hadoop.mapred.TextInputFormat         
-OutputFormat:          
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat       
-Compressed:            No                       
-Num Buckets:           -1                       
-Bucket Columns:        []                       
-Sort Columns:          []                       
-Storage Desc Params:            
-       serialization.format    1                   
-PREHOOK: query: describe formatted dummy partition (hr=1994)
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@dummy
-POSTHOOK: query: describe formatted dummy partition (hr=1994)
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@dummy
-# col_name             data_type               comment             
-                
-key                    int                                         
-                
-# Partition Information                 
-# col_name             data_type               comment             
-                
-hr                     int                                         
-                
-# Detailed Partition Information                
-Partition Value:       [1994]                   
-Database:              default                  
-Table:                 dummy                    
-#### A masked pattern was here ####
-Partition Parameters:           
-       COLUMN_STATS_ACCURATE   true                
-       numFiles                1                   
-       numRows                 1                   
-       rawDataSize             1                   
-       totalSize               2                   
-#### A masked pattern was here ####
-                
-# Storage Information           
-SerDe Library:         org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe      
 
-InputFormat:           org.apache.hadoop.mapred.TextInputFormat         
-OutputFormat:          
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat       
-Compressed:            No                       
-Num Buckets:           -1                       
-Bucket Columns:        []                       
-Sort Columns:          []                       
-Storage Desc Params:            
-       serialization.format    1                   
-PREHOOK: query: describe formatted dummy partition (hr=1998)
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@dummy
-POSTHOOK: query: describe formatted dummy partition (hr=1998)
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@dummy
-# col_name             data_type               comment             
-                
-key                    int                                         
-                
-# Partition Information                 
-# col_name             data_type               comment             
-                
-hr                     int                                         
-                
-# Detailed Partition Information                
-Partition Value:       [1998]                   
-Database:              default                  
-Table:                 dummy                    
-#### A masked pattern was here ####
-Partition Parameters:           
-       COLUMN_STATS_ACCURATE   true                
-       numFiles                1                   
-       numRows                 2                   
-       rawDataSize             2                   
-       totalSize               4                   
-#### A masked pattern was here ####
-                
-# Storage Information           
-SerDe Library:         org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe      
 
-InputFormat:           org.apache.hadoop.mapred.TextInputFormat         
-OutputFormat:          
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat       
-Compressed:            No                       
-Num Buckets:           -1                       
-Bucket Columns:        []                       
-Sort Columns:          []                       
-Storage Desc Params:            
-       serialization.format    1                   
-PREHOOK: query: describe formatted dummy partition (hr=1996)
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@dummy
-POSTHOOK: query: describe formatted dummy partition (hr=1996)
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@dummy
-# col_name             data_type               comment             
-                
-key                    int                                         
-                
-# Partition Information                 
-# col_name             data_type               comment             
-                
-hr                     int                                         
-                
-# Detailed Partition Information                
-Partition Value:       [1996]                   
-Database:              default                  
-Table:                 dummy                    
-#### A masked pattern was here ####
-Partition Parameters:           
-       COLUMN_STATS_ACCURATE   true                
-       numFiles                1                   
-       numRows                 1                   
-       rawDataSize             1                   
-       totalSize               2                   
-#### A masked pattern was here ####
-                
-# Storage Information           
-SerDe Library:         org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe      
 
-InputFormat:           org.apache.hadoop.mapred.TextInputFormat         
-OutputFormat:          
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat       
-Compressed:            No                       
-Num Buckets:           -1                       
-Bucket Columns:        []                       
-Sort Columns:          []                       
-Storage Desc Params:            
-       serialization.format    1                   
-PREHOOK: query: drop table tbl
-PREHOOK: type: DROPTABLE
-PREHOOK: Input: default@tbl
-PREHOOK: Output: default@tbl
-POSTHOOK: query: drop table tbl
-POSTHOOK: type: DROPTABLE
-POSTHOOK: Input: default@tbl
-POSTHOOK: Output: default@tbl
-PREHOOK: query: drop table dummy
-PREHOOK: type: DROPTABLE
-PREHOOK: Input: default@dummy
-PREHOOK: Output: default@dummy
-POSTHOOK: query: drop table dummy
-POSTHOOK: type: DROPTABLE
-POSTHOOK: Input: default@dummy
-POSTHOOK: Output: default@dummy

http://git-wip-us.apache.org/repos/asf/hive/blob/884ff9ca/ql/src/test/results/clientpositive/spark/stats_counter.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/stats_counter.q.out 
b/ql/src/test/results/clientpositive/spark/stats_counter.q.out
deleted file mode 100644
index 8b3dcea..0000000
--- a/ql/src/test/results/clientpositive/spark/stats_counter.q.out
+++ /dev/null
@@ -1,102 +0,0 @@
-PREHOOK: query: -- by analyze
-create table dummy1 as select * from src
-PREHOOK: type: CREATETABLE_AS_SELECT
-PREHOOK: Input: default@src
-PREHOOK: Output: database:default
-PREHOOK: Output: default@dummy1
-POSTHOOK: query: -- by analyze
-create table dummy1 as select * from src
-POSTHOOK: type: CREATETABLE_AS_SELECT
-POSTHOOK: Input: default@src
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@dummy1
-PREHOOK: query: analyze table dummy1 compute statistics
-PREHOOK: type: QUERY
-PREHOOK: Input: default@dummy1
-PREHOOK: Output: default@dummy1
-POSTHOOK: query: analyze table dummy1 compute statistics
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@dummy1
-POSTHOOK: Output: default@dummy1
-PREHOOK: query: desc formatted dummy1
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@dummy1
-POSTHOOK: query: desc formatted dummy1
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@dummy1
-# col_name             data_type               comment             
-                
-key                    string                                      
-value                  string                                      
-                
-# Detailed Table Information            
-Database:              default                  
-#### A masked pattern was here ####
-Retention:             0                        
-#### A masked pattern was here ####
-Table Type:            MANAGED_TABLE            
-Table Parameters:               
-       COLUMN_STATS_ACCURATE   true                
-       numFiles                1                   
-       numRows                 500                 
-       rawDataSize             5312                
-       totalSize               5812                
-#### A masked pattern was here ####
-                
-# Storage Information           
-SerDe Library:         org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe      
 
-InputFormat:           org.apache.hadoop.mapred.TextInputFormat         
-OutputFormat:          
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat       
-Compressed:            No                       
-Num Buckets:           -1                       
-Bucket Columns:        []                       
-Sort Columns:          []                       
-Storage Desc Params:            
-       serialization.format    1                   
-PREHOOK: query: -- by autogather
-create table dummy2 as select * from src
-PREHOOK: type: CREATETABLE_AS_SELECT
-PREHOOK: Input: default@src
-PREHOOK: Output: database:default
-PREHOOK: Output: default@dummy2
-POSTHOOK: query: -- by autogather
-create table dummy2 as select * from src
-POSTHOOK: type: CREATETABLE_AS_SELECT
-POSTHOOK: Input: default@src
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@dummy2
-PREHOOK: query: desc formatted dummy2
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@dummy2
-POSTHOOK: query: desc formatted dummy2
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@dummy2
-# col_name             data_type               comment             
-                
-key                    string                                      
-value                  string                                      
-                
-# Detailed Table Information            
-Database:              default                  
-#### A masked pattern was here ####
-Retention:             0                        
-#### A masked pattern was here ####
-Table Type:            MANAGED_TABLE            
-Table Parameters:               
-       COLUMN_STATS_ACCURATE   true                
-       numFiles                1                   
-       numRows                 500                 
-       rawDataSize             5312                
-       totalSize               5812                
-#### A masked pattern was here ####
-                
-# Storage Information           
-SerDe Library:         org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe      
 
-InputFormat:           org.apache.hadoop.mapred.TextInputFormat         
-OutputFormat:          
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat       
-Compressed:            No                       
-Num Buckets:           -1                       
-Bucket Columns:        []                       
-Sort Columns:          []                       
-Storage Desc Params:            
-       serialization.format    1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/884ff9ca/ql/src/test/results/clientpositive/spark/stats_counter_partitioned.q.out
----------------------------------------------------------------------
diff --git 
a/ql/src/test/results/clientpositive/spark/stats_counter_partitioned.q.out 
b/ql/src/test/results/clientpositive/spark/stats_counter_partitioned.q.out
deleted file mode 100644
index 12e1fbe..0000000
--- a/ql/src/test/results/clientpositive/spark/stats_counter_partitioned.q.out
+++ /dev/null
@@ -1,465 +0,0 @@
-PREHOOK: query: -- partitioned table analyze 
-
-create table dummy (key string, value string) partitioned by (ds string, hr 
string)
-PREHOOK: type: CREATETABLE
-PREHOOK: Output: database:default
-PREHOOK: Output: default@dummy
-POSTHOOK: query: -- partitioned table analyze 
-
-create table dummy (key string, value string) partitioned by (ds string, hr 
string)
-POSTHOOK: type: CREATETABLE
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@dummy
-PREHOOK: query: load data local inpath '../../data/files/kv1.txt' into table 
dummy partition (ds='2008',hr='12')
-PREHOOK: type: LOAD
-#### A masked pattern was here ####
-PREHOOK: Output: default@dummy
-POSTHOOK: query: load data local inpath '../../data/files/kv1.txt' into table 
dummy partition (ds='2008',hr='12')
-POSTHOOK: type: LOAD
-#### A masked pattern was here ####
-POSTHOOK: Output: default@dummy
-POSTHOOK: Output: default@dummy@ds=2008/hr=12
-PREHOOK: query: load data local inpath '../../data/files/kv1.txt' into table 
dummy partition (ds='2008',hr='11')
-PREHOOK: type: LOAD
-#### A masked pattern was here ####
-PREHOOK: Output: default@dummy
-POSTHOOK: query: load data local inpath '../../data/files/kv1.txt' into table 
dummy partition (ds='2008',hr='11')
-POSTHOOK: type: LOAD
-#### A masked pattern was here ####
-POSTHOOK: Output: default@dummy
-POSTHOOK: Output: default@dummy@ds=2008/hr=11
-PREHOOK: query: analyze table dummy partition (ds,hr) compute statistics
-PREHOOK: type: QUERY
-PREHOOK: Input: default@dummy
-PREHOOK: Input: default@dummy@ds=2008/hr=11
-PREHOOK: Input: default@dummy@ds=2008/hr=12
-PREHOOK: Output: default@dummy
-PREHOOK: Output: default@dummy@ds=2008/hr=11
-PREHOOK: Output: default@dummy@ds=2008/hr=12
-POSTHOOK: query: analyze table dummy partition (ds,hr) compute statistics
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@dummy
-POSTHOOK: Input: default@dummy@ds=2008/hr=11
-POSTHOOK: Input: default@dummy@ds=2008/hr=12
-POSTHOOK: Output: default@dummy
-POSTHOOK: Output: default@dummy@ds=2008/hr=11
-POSTHOOK: Output: default@dummy@ds=2008/hr=12
-PREHOOK: query: describe formatted dummy partition (ds='2008', hr='11')
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@dummy
-POSTHOOK: query: describe formatted dummy partition (ds='2008', hr='11')
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@dummy
-# col_name             data_type               comment             
-                
-key                    string                                      
-value                  string                                      
-                
-# Partition Information                 
-# col_name             data_type               comment             
-                
-ds                     string                                      
-hr                     string                                      
-                
-# Detailed Partition Information                
-Partition Value:       [2008, 11]               
-Database:              default                  
-Table:                 dummy                    
-#### A masked pattern was here ####
-Partition Parameters:           
-       COLUMN_STATS_ACCURATE   true                
-       numFiles                1                   
-       numRows                 500                 
-       rawDataSize             5312                
-       totalSize               5812                
-#### A masked pattern was here ####
-                
-# Storage Information           
-SerDe Library:         org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe      
 
-InputFormat:           org.apache.hadoop.mapred.TextInputFormat         
-OutputFormat:          
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat       
-Compressed:            No                       
-Num Buckets:           -1                       
-Bucket Columns:        []                       
-Sort Columns:          []                       
-Storage Desc Params:            
-       serialization.format    1                   
-PREHOOK: query: describe formatted dummy partition (ds='2008', hr='12')
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@dummy
-POSTHOOK: query: describe formatted dummy partition (ds='2008', hr='12')
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@dummy
-# col_name             data_type               comment             
-                
-key                    string                                      
-value                  string                                      
-                
-# Partition Information                 
-# col_name             data_type               comment             
-                
-ds                     string                                      
-hr                     string                                      
-                
-# Detailed Partition Information                
-Partition Value:       [2008, 12]               
-Database:              default                  
-Table:                 dummy                    
-#### A masked pattern was here ####
-Partition Parameters:           
-       COLUMN_STATS_ACCURATE   true                
-       numFiles                1                   
-       numRows                 500                 
-       rawDataSize             5312                
-       totalSize               5812                
-#### A masked pattern was here ####
-                
-# Storage Information           
-SerDe Library:         org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe      
 
-InputFormat:           org.apache.hadoop.mapred.TextInputFormat         
-OutputFormat:          
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat       
-Compressed:            No                       
-Num Buckets:           -1                       
-Bucket Columns:        []                       
-Sort Columns:          []                       
-Storage Desc Params:            
-       serialization.format    1                   
-PREHOOK: query: drop table dummy
-PREHOOK: type: DROPTABLE
-PREHOOK: Input: default@dummy
-PREHOOK: Output: default@dummy
-POSTHOOK: query: drop table dummy
-POSTHOOK: type: DROPTABLE
-POSTHOOK: Input: default@dummy
-POSTHOOK: Output: default@dummy
-PREHOOK: query: -- static partitioned table on insert
-
-create table dummy (key string, value string) partitioned by (ds string, hr 
string)
-PREHOOK: type: CREATETABLE
-PREHOOK: Output: database:default
-PREHOOK: Output: default@dummy
-POSTHOOK: query: -- static partitioned table on insert
-
-create table dummy (key string, value string) partitioned by (ds string, hr 
string)
-POSTHOOK: type: CREATETABLE
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@dummy
-PREHOOK: query: insert overwrite table dummy partition (ds='10',hr='11') 
select * from src
-PREHOOK: type: QUERY
-PREHOOK: Input: default@src
-PREHOOK: Output: default@dummy@ds=10/hr=11
-POSTHOOK: query: insert overwrite table dummy partition (ds='10',hr='11') 
select * from src
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@src
-POSTHOOK: Output: default@dummy@ds=10/hr=11
-POSTHOOK: Lineage: dummy PARTITION(ds=10,hr=11).key SIMPLE 
[(src)src.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: dummy PARTITION(ds=10,hr=11).value SIMPLE 
[(src)src.FieldSchema(name:value, type:string, comment:default), ]
-PREHOOK: query: insert overwrite table dummy partition (ds='10',hr='12') 
select * from src
-PREHOOK: type: QUERY
-PREHOOK: Input: default@src
-PREHOOK: Output: default@dummy@ds=10/hr=12
-POSTHOOK: query: insert overwrite table dummy partition (ds='10',hr='12') 
select * from src
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@src
-POSTHOOK: Output: default@dummy@ds=10/hr=12
-POSTHOOK: Lineage: dummy PARTITION(ds=10,hr=12).key SIMPLE 
[(src)src.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: dummy PARTITION(ds=10,hr=12).value SIMPLE 
[(src)src.FieldSchema(name:value, type:string, comment:default), ]
-PREHOOK: query: describe formatted dummy partition (ds='10', hr='11')
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@dummy
-POSTHOOK: query: describe formatted dummy partition (ds='10', hr='11')
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@dummy
-# col_name             data_type               comment             
-                
-key                    string                                      
-value                  string                                      
-                
-# Partition Information                 
-# col_name             data_type               comment             
-                
-ds                     string                                      
-hr                     string                                      
-                
-# Detailed Partition Information                
-Partition Value:       [10, 11]                 
-Database:              default                  
-Table:                 dummy                    
-#### A masked pattern was here ####
-Partition Parameters:           
-       COLUMN_STATS_ACCURATE   true                
-       numFiles                1                   
-       numRows                 500                 
-       rawDataSize             5312                
-       totalSize               5812                
-#### A masked pattern was here ####
-                
-# Storage Information           
-SerDe Library:         org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe      
 
-InputFormat:           org.apache.hadoop.mapred.TextInputFormat         
-OutputFormat:          
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat       
-Compressed:            No                       
-Num Buckets:           -1                       
-Bucket Columns:        []                       
-Sort Columns:          []                       
-Storage Desc Params:            
-       serialization.format    1                   
-PREHOOK: query: describe formatted dummy partition (ds='10', hr='12')
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@dummy
-POSTHOOK: query: describe formatted dummy partition (ds='10', hr='12')
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@dummy
-# col_name             data_type               comment             
-                
-key                    string                                      
-value                  string                                      
-                
-# Partition Information                 
-# col_name             data_type               comment             
-                
-ds                     string                                      
-hr                     string                                      
-                
-# Detailed Partition Information                
-Partition Value:       [10, 12]                 
-Database:              default                  
-Table:                 dummy                    
-#### A masked pattern was here ####
-Partition Parameters:           
-       COLUMN_STATS_ACCURATE   true                
-       numFiles                1                   
-       numRows                 500                 
-       rawDataSize             5312                
-       totalSize               5812                
-#### A masked pattern was here ####
-                
-# Storage Information           
-SerDe Library:         org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe      
 
-InputFormat:           org.apache.hadoop.mapred.TextInputFormat         
-OutputFormat:          
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat       
-Compressed:            No                       
-Num Buckets:           -1                       
-Bucket Columns:        []                       
-Sort Columns:          []                       
-Storage Desc Params:            
-       serialization.format    1                   
-PREHOOK: query: drop table dummy
-PREHOOK: type: DROPTABLE
-PREHOOK: Input: default@dummy
-PREHOOK: Output: default@dummy
-POSTHOOK: query: drop table dummy
-POSTHOOK: type: DROPTABLE
-POSTHOOK: Input: default@dummy
-POSTHOOK: Output: default@dummy
-PREHOOK: query: -- dynamic partitioned table on insert
-
-create table dummy (key int) partitioned by (hr int)
-PREHOOK: type: CREATETABLE
-PREHOOK: Output: database:default
-PREHOOK: Output: default@dummy
-POSTHOOK: query: -- dynamic partitioned table on insert
-
-create table dummy (key int) partitioned by (hr int)
-POSTHOOK: type: CREATETABLE
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@dummy
-PREHOOK: query: CREATE TABLE tbl(key int, value int) ROW FORMAT DELIMITED 
FIELDS TERMINATED BY '|'
-PREHOOK: type: CREATETABLE
-PREHOOK: Output: database:default
-PREHOOK: Output: default@tbl
-POSTHOOK: query: CREATE TABLE tbl(key int, value int) ROW FORMAT DELIMITED 
FIELDS TERMINATED BY '|'
-POSTHOOK: type: CREATETABLE
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@tbl
-PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/tbl.txt' OVERWRITE 
INTO TABLE tbl
-PREHOOK: type: LOAD
-#### A masked pattern was here ####
-PREHOOK: Output: default@tbl
-POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/tbl.txt' OVERWRITE 
INTO TABLE tbl
-POSTHOOK: type: LOAD
-#### A masked pattern was here ####
-POSTHOOK: Output: default@tbl
-PREHOOK: query: insert overwrite table dummy partition (hr) select * from tbl
-PREHOOK: type: QUERY
-PREHOOK: Input: default@tbl
-PREHOOK: Output: default@dummy
-POSTHOOK: query: insert overwrite table dummy partition (hr) select * from tbl
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@tbl
-POSTHOOK: Output: default@dummy@hr=1994
-POSTHOOK: Output: default@dummy@hr=1996
-POSTHOOK: Output: default@dummy@hr=1997
-POSTHOOK: Output: default@dummy@hr=1998
-POSTHOOK: Lineage: dummy PARTITION(hr=1994).key SIMPLE 
[(tbl)tbl.FieldSchema(name:key, type:int, comment:null), ]
-POSTHOOK: Lineage: dummy PARTITION(hr=1996).key SIMPLE 
[(tbl)tbl.FieldSchema(name:key, type:int, comment:null), ]
-POSTHOOK: Lineage: dummy PARTITION(hr=1997).key SIMPLE 
[(tbl)tbl.FieldSchema(name:key, type:int, comment:null), ]
-POSTHOOK: Lineage: dummy PARTITION(hr=1998).key SIMPLE 
[(tbl)tbl.FieldSchema(name:key, type:int, comment:null), ]
-PREHOOK: query: describe formatted dummy partition (hr=1997)
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@dummy
-POSTHOOK: query: describe formatted dummy partition (hr=1997)
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@dummy
-# col_name             data_type               comment             
-                
-key                    int                                         
-                
-# Partition Information                 
-# col_name             data_type               comment             
-                
-hr                     int                                         
-                
-# Detailed Partition Information                
-Partition Value:       [1997]                   
-Database:              default                  
-Table:                 dummy                    
-#### A masked pattern was here ####
-Partition Parameters:           
-       COLUMN_STATS_ACCURATE   true                
-       numFiles                1                   
-       numRows                 0                   
-       rawDataSize             0                   
-       totalSize               12                  
-#### A masked pattern was here ####
-                
-# Storage Information           
-SerDe Library:         org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe      
 
-InputFormat:           org.apache.hadoop.mapred.TextInputFormat         
-OutputFormat:          
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat       
-Compressed:            No                       
-Num Buckets:           -1                       
-Bucket Columns:        []                       
-Sort Columns:          []                       
-Storage Desc Params:            
-       serialization.format    1                   
-PREHOOK: query: describe formatted dummy partition (hr=1994)
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@dummy
-POSTHOOK: query: describe formatted dummy partition (hr=1994)
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@dummy
-# col_name             data_type               comment             
-                
-key                    int                                         
-                
-# Partition Information                 
-# col_name             data_type               comment             
-                
-hr                     int                                         
-                
-# Detailed Partition Information                
-Partition Value:       [1994]                   
-Database:              default                  
-Table:                 dummy                    
-#### A masked pattern was here ####
-Partition Parameters:           
-       COLUMN_STATS_ACCURATE   true                
-       numFiles                1                   
-       numRows                 0                   
-       rawDataSize             0                   
-       totalSize               2                   
-#### A masked pattern was here ####
-                
-# Storage Information           
-SerDe Library:         org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe      
 
-InputFormat:           org.apache.hadoop.mapred.TextInputFormat         
-OutputFormat:          
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat       
-Compressed:            No                       
-Num Buckets:           -1                       
-Bucket Columns:        []                       
-Sort Columns:          []                       
-Storage Desc Params:            
-       serialization.format    1                   
-PREHOOK: query: describe formatted dummy partition (hr=1998)
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@dummy
-POSTHOOK: query: describe formatted dummy partition (hr=1998)
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@dummy
-# col_name             data_type               comment             
-                
-key                    int                                         
-                
-# Partition Information                 
-# col_name             data_type               comment             
-                
-hr                     int                                         
-                
-# Detailed Partition Information                
-Partition Value:       [1998]                   
-Database:              default                  
-Table:                 dummy                    
-#### A masked pattern was here ####
-Partition Parameters:           
-       COLUMN_STATS_ACCURATE   true                
-       numFiles                1                   
-       numRows                 0                   
-       rawDataSize             0                   
-       totalSize               4                   
-#### A masked pattern was here ####
-                
-# Storage Information           
-SerDe Library:         org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe      
 
-InputFormat:           org.apache.hadoop.mapred.TextInputFormat         
-OutputFormat:          
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat       
-Compressed:            No                       
-Num Buckets:           -1                       
-Bucket Columns:        []                       
-Sort Columns:          []                       
-Storage Desc Params:            
-       serialization.format    1                   
-PREHOOK: query: describe formatted dummy partition (hr=1996)
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@dummy
-POSTHOOK: query: describe formatted dummy partition (hr=1996)
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@dummy
-# col_name             data_type               comment             
-                
-key                    int                                         
-                
-# Partition Information                 
-# col_name             data_type               comment             
-                
-hr                     int                                         
-                
-# Detailed Partition Information                
-Partition Value:       [1996]                   
-Database:              default                  
-Table:                 dummy                    
-#### A masked pattern was here ####
-Partition Parameters:           
-       COLUMN_STATS_ACCURATE   true                
-       numFiles                1                   
-       numRows                 0                   
-       rawDataSize             0                   
-       totalSize               2                   
-#### A masked pattern was here ####
-                
-# Storage Information           
-SerDe Library:         org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe      
 
-InputFormat:           org.apache.hadoop.mapred.TextInputFormat         
-OutputFormat:          
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat       
-Compressed:            No                       
-Num Buckets:           -1                       
-Bucket Columns:        []                       
-Sort Columns:          []                       
-Storage Desc Params:            
-       serialization.format    1                   
-PREHOOK: query: drop table tbl
-PREHOOK: type: DROPTABLE
-PREHOOK: Input: default@tbl
-PREHOOK: Output: default@tbl
-POSTHOOK: query: drop table tbl
-POSTHOOK: type: DROPTABLE
-POSTHOOK: Input: default@tbl
-POSTHOOK: Output: default@tbl
-PREHOOK: query: drop table dummy
-PREHOOK: type: DROPTABLE
-PREHOOK: Input: default@dummy
-PREHOOK: Output: default@dummy
-POSTHOOK: query: drop table dummy
-POSTHOOK: type: DROPTABLE
-POSTHOOK: Input: default@dummy
-POSTHOOK: Output: default@dummy

http://git-wip-us.apache.org/repos/asf/hive/blob/884ff9ca/ql/src/test/results/clientpositive/stats_counter.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/stats_counter.q.out 
b/ql/src/test/results/clientpositive/stats_counter.q.out
deleted file mode 100644
index 8b3dcea..0000000
--- a/ql/src/test/results/clientpositive/stats_counter.q.out
+++ /dev/null
@@ -1,102 +0,0 @@
-PREHOOK: query: -- by analyze
-create table dummy1 as select * from src
-PREHOOK: type: CREATETABLE_AS_SELECT
-PREHOOK: Input: default@src
-PREHOOK: Output: database:default
-PREHOOK: Output: default@dummy1
-POSTHOOK: query: -- by analyze
-create table dummy1 as select * from src
-POSTHOOK: type: CREATETABLE_AS_SELECT
-POSTHOOK: Input: default@src
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@dummy1
-PREHOOK: query: analyze table dummy1 compute statistics
-PREHOOK: type: QUERY
-PREHOOK: Input: default@dummy1
-PREHOOK: Output: default@dummy1
-POSTHOOK: query: analyze table dummy1 compute statistics
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@dummy1
-POSTHOOK: Output: default@dummy1
-PREHOOK: query: desc formatted dummy1
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@dummy1
-POSTHOOK: query: desc formatted dummy1
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@dummy1
-# col_name             data_type               comment             
-                
-key                    string                                      
-value                  string                                      
-                
-# Detailed Table Information            
-Database:              default                  
-#### A masked pattern was here ####
-Retention:             0                        
-#### A masked pattern was here ####
-Table Type:            MANAGED_TABLE            
-Table Parameters:               
-       COLUMN_STATS_ACCURATE   true                
-       numFiles                1                   
-       numRows                 500                 
-       rawDataSize             5312                
-       totalSize               5812                
-#### A masked pattern was here ####
-                
-# Storage Information           
-SerDe Library:         org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe      
 
-InputFormat:           org.apache.hadoop.mapred.TextInputFormat         
-OutputFormat:          
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat       
-Compressed:            No                       
-Num Buckets:           -1                       
-Bucket Columns:        []                       
-Sort Columns:          []                       
-Storage Desc Params:            
-       serialization.format    1                   
-PREHOOK: query: -- by autogather
-create table dummy2 as select * from src
-PREHOOK: type: CREATETABLE_AS_SELECT
-PREHOOK: Input: default@src
-PREHOOK: Output: database:default
-PREHOOK: Output: default@dummy2
-POSTHOOK: query: -- by autogather
-create table dummy2 as select * from src
-POSTHOOK: type: CREATETABLE_AS_SELECT
-POSTHOOK: Input: default@src
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@dummy2
-PREHOOK: query: desc formatted dummy2
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@dummy2
-POSTHOOK: query: desc formatted dummy2
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@dummy2
-# col_name             data_type               comment             
-                
-key                    string                                      
-value                  string                                      
-                
-# Detailed Table Information            
-Database:              default                  
-#### A masked pattern was here ####
-Retention:             0                        
-#### A masked pattern was here ####
-Table Type:            MANAGED_TABLE            
-Table Parameters:               
-       COLUMN_STATS_ACCURATE   true                
-       numFiles                1                   
-       numRows                 500                 
-       rawDataSize             5312                
-       totalSize               5812                
-#### A masked pattern was here ####
-                
-# Storage Information           
-SerDe Library:         org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe      
 
-InputFormat:           org.apache.hadoop.mapred.TextInputFormat         
-OutputFormat:          
org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat       
-Compressed:            No                       
-Num Buckets:           -1                       
-Bucket Columns:        []                       
-Sort Columns:          []                       
-Storage Desc Params:            
-       serialization.format    1                   

Reply via email to