You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by kg...@apache.org on 2018/06/28 09:45:25 UTC
[9/9] hive git commit: HIVE-18140: Partitioned tables statistics can
go wrong in basic stats mixed case (Zoltan Haindrich reviewed by Ashutosh
Chauhan)
HIVE-18140: Partitioned tables statistics can go wrong in basic stats mixed case (Zoltan Haindrich reviewed by Ashutosh Chauhan)
Signed-off-by: Zoltan Haindrich <ki...@rxd.hu>
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/15d7d6dc
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/15d7d6dc
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/15d7d6dc
Branch: refs/heads/master
Commit: 15d7d6dc97fae5e7502c5ec869c2aedc7170ca2c
Parents: 67b0a67
Author: Zoltan Haindrich <ki...@rxd.hu>
Authored: Thu Jun 28 11:43:10 2018 +0200
Committer: Zoltan Haindrich <ki...@rxd.hu>
Committed: Thu Jun 28 11:43:10 2018 +0200
----------------------------------------------------------------------
itests/qtest/pom.xml | 1 -
.../ql/optimizer/calcite/RelOptHiveTable.java | 4 +-
.../apache/hadoop/hive/ql/plan/Statistics.java | 12 +-
.../apache/hadoop/hive/ql/stats/BasicStats.java | 325 +++++++++++++++++++
.../apache/hadoop/hive/ql/stats/Partish.java | 13 +
.../apache/hadoop/hive/ql/stats/StatsUtils.java | 265 +++++----------
.../hadoop/hive/ql/stats/TestBasicStats.java | 123 +++++++
ql/src/test/queries/clientpositive/stats8.q | 2 +
.../bucket_mapjoin_mismatch1.q.out | 36 +-
.../clientpositive/acid_table_stats.q.out | 12 +-
.../analyze_table_null_partition.q.out | 4 +-
.../clientpositive/annotate_stats_part.q.out | 12 +-
.../clientpositive/autoColumnStats_2.q.out | 4 +-
.../clientpositive/beeline/smb_mapjoin_10.q.out | 4 +-
.../clientpositive/bucket_map_join_spark1.q.out | 56 ++--
.../clientpositive/bucket_map_join_spark2.q.out | 56 ++--
.../clientpositive/bucket_map_join_spark3.q.out | 56 ++--
.../clientpositive/bucketcontext_1.q.out | 22 +-
.../clientpositive/bucketcontext_2.q.out | 22 +-
.../clientpositive/bucketcontext_3.q.out | 22 +-
.../clientpositive/bucketcontext_4.q.out | 22 +-
.../clientpositive/bucketcontext_6.q.out | 18 +-
.../clientpositive/bucketcontext_7.q.out | 22 +-
.../clientpositive/bucketcontext_8.q.out | 22 +-
.../clientpositive/bucketmapjoin10.q.out | 18 +-
.../clientpositive/bucketmapjoin11.q.out | 36 +-
.../clientpositive/bucketmapjoin12.q.out | 36 +-
.../results/clientpositive/bucketmapjoin5.q.out | 40 +--
.../results/clientpositive/bucketmapjoin8.q.out | 36 +-
.../results/clientpositive/bucketmapjoin9.q.out | 36 +-
.../clientpositive/bucketmapjoin_negative.q.out | 20 +-
.../bucketmapjoin_negative2.q.out | 20 +-
.../clientpositive/columnstats_partlvl.q.out | 56 ++--
.../clientpositive/columnstats_partlvl_dp.q.out | 42 +--
.../insert2_overwrite_partitions.q.out | 48 +--
.../list_bucket_query_oneskew_3.q.out | 6 +-
.../llap/auto_sortmerge_join_1.q.out | 66 ++--
.../llap/auto_sortmerge_join_11.q.out | 88 ++---
.../llap/auto_sortmerge_join_12.q.out | 38 +--
.../llap/auto_sortmerge_join_2.q.out | 44 +--
.../llap/auto_sortmerge_join_3.q.out | 66 ++--
.../llap/auto_sortmerge_join_4.q.out | 66 ++--
.../llap/auto_sortmerge_join_7.q.out | 66 ++--
.../llap/auto_sortmerge_join_8.q.out | 66 ++--
.../clientpositive/llap/bucketmapjoin1.q.out | 48 +--
.../clientpositive/llap/bucketmapjoin2.q.out | 96 +++---
.../clientpositive/llap/bucketmapjoin3.q.out | 64 ++--
.../clientpositive/llap/bucketmapjoin7.q.out | 24 +-
.../llap/column_table_stats.q.out | 42 +--
.../llap/dynpart_sort_optimization_acid.q.out | 144 ++++----
.../llap/insert1_overwrite_partitions.q.out | 70 ++--
.../insert_values_orig_table_use_metadata.q.out | 12 +-
.../llap/join_reordering_no_stats.q.out | 80 ++---
.../results/clientpositive/llap/stats11.q.out | 48 +--
.../merge_dynamic_partition.q.out | 54 +--
.../merge_dynamic_partition2.q.out | 18 +-
.../merge_dynamic_partition3.q.out | 18 +-
.../results/clientpositive/nullgroup5.q.out | 4 +-
.../test/results/clientpositive/row__id.q.out | 18 +-
.../results/clientpositive/smb_mapjoin_10.q.out | 4 +-
.../spark/auto_sortmerge_join_1.q.out | 48 +--
.../spark/auto_sortmerge_join_12.q.out | 44 +--
.../spark/auto_sortmerge_join_2.q.out | 32 +-
.../spark/auto_sortmerge_join_3.q.out | 48 +--
.../spark/auto_sortmerge_join_4.q.out | 48 +--
.../spark/auto_sortmerge_join_7.q.out | 48 +--
.../spark/auto_sortmerge_join_8.q.out | 48 +--
.../spark/bucket_map_join_spark1.q.out | 36 +-
.../spark/bucket_map_join_spark2.q.out | 36 +-
.../spark/bucket_map_join_spark3.q.out | 36 +-
.../clientpositive/spark/bucketmapjoin1.q.out | 28 +-
.../clientpositive/spark/bucketmapjoin10.q.out | 18 +-
.../clientpositive/spark/bucketmapjoin11.q.out | 36 +-
.../clientpositive/spark/bucketmapjoin12.q.out | 36 +-
.../clientpositive/spark/bucketmapjoin2.q.out | 66 ++--
.../clientpositive/spark/bucketmapjoin3.q.out | 44 +--
.../clientpositive/spark/bucketmapjoin5.q.out | 20 +-
.../clientpositive/spark/bucketmapjoin7.q.out | 20 +-
.../spark/bucketmapjoin7.q.out_spark | 20 +-
.../clientpositive/spark/bucketmapjoin8.q.out | 36 +-
.../clientpositive/spark/bucketmapjoin9.q.out | 36 +-
.../spark/bucketmapjoin_negative.q.out | 10 +-
.../spark/bucketmapjoin_negative2.q.out | 10 +-
.../clientpositive/spark/smb_mapjoin_10.q.out | 10 +-
.../results/clientpositive/spark/stats12.q.out | 2 +-
.../results/clientpositive/spark/stats13.q.out | 2 +-
.../results/clientpositive/spark/stats2.q.out | 2 +-
.../results/clientpositive/spark/stats7.q.out | 2 +-
.../results/clientpositive/spark/stats8.q.out | 78 ++++-
.../test/results/clientpositive/stats12.q.out | 2 +-
.../test/results/clientpositive/stats13.q.out | 2 +-
ql/src/test/results/clientpositive/stats2.q.out | 2 +-
ql/src/test/results/clientpositive/stats7.q.out | 2 +-
ql/src/test/results/clientpositive/stats8.q.out | 78 ++++-
94 files changed, 2129 insertions(+), 1630 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/15d7d6dc/itests/qtest/pom.xml
----------------------------------------------------------------------
diff --git a/itests/qtest/pom.xml b/itests/qtest/pom.xml
index ce08126..a1400fd 100644
--- a/itests/qtest/pom.xml
+++ b/itests/qtest/pom.xml
@@ -450,7 +450,6 @@
<configuration>
<target>
<property name="test.classpath" refid="maven.test.classpath"/>
- <echo message="${test.classpath}"/>
<taskdef resource="net/sf/antcontrib/antcontrib.properties"
classpathref="maven.plugin.classpath" />
<mkdir dir="${project.build.directory}/qfile-results/clientpositive/" />
http://git-wip-us.apache.org/repos/asf/hive/blob/15d7d6dc/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/RelOptHiveTable.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/RelOptHiveTable.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/RelOptHiveTable.java
index e5e475e..6cc6d02 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/RelOptHiveTable.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/RelOptHiveTable.java
@@ -95,9 +95,7 @@ public class RelOptHiveTable extends RelOptAbstractTable {
Map<String, ColumnStatsList> colStatsCache;
AtomicInteger noColsMissingStats;
- protected static final Logger LOG = LoggerFactory
- .getLogger(RelOptHiveTable.class
- .getName());
+ protected static final Logger LOG = LoggerFactory.getLogger(RelOptHiveTable.class.getName());
public RelOptHiveTable(RelOptSchema calciteSchema, String qualifiedTblName,
RelDataType rowType, Table hiveTblMetadata, List<ColumnInfo> hiveNonPartitionCols,
http://git-wip-us.apache.org/repos/asf/hive/blob/15d7d6dc/ql/src/java/org/apache/hadoop/hive/ql/plan/Statistics.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/Statistics.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/Statistics.java
index fd461ae..6babe49 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/Statistics.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/Statistics.java
@@ -38,9 +38,16 @@ public class Statistics implements Serializable {
public enum State {
NONE, PARTIAL, COMPLETE;
- boolean morePreciseThan(State other) {
+ public boolean morePreciseThan(State other) {
return ordinal() >= other.ordinal();
}
+
+ public State merge(State otherState) {
+ if (this == otherState) {
+ return this;
+ }
+ return PARTIAL;
+ }
}
private long numRows;
@@ -313,8 +320,7 @@ public class Statistics implements Serializable {
}
public Statistics scaleToRowCount(long newRowCount, boolean downScaleOnly) {
- Statistics ret;
- ret = clone();
+ Statistics ret = clone();
if (numRows == 0) {
return ret;
}
http://git-wip-us.apache.org/repos/asf/hive/blob/15d7d6dc/ql/src/java/org/apache/hadoop/hive/ql/stats/BasicStats.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/stats/BasicStats.java b/ql/src/java/org/apache/hadoop/hive/ql/stats/BasicStats.java
new file mode 100644
index 0000000..b723820
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/stats/BasicStats.java
@@ -0,0 +1,325 @@
+/**
+ * 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 java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.ql.plan.Statistics;
+import org.apache.hadoop.hive.ql.plan.Statistics.State;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Lists;
+import com.google.common.util.concurrent.MoreExecutors;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+public class BasicStats {
+
+ private static final Logger LOG = LoggerFactory.getLogger(BasicStats.class.getName());
+
+ public static class Factory {
+
+ private final List<IStatsEnhancer> enhancers = new LinkedList<>();
+
+ public Factory(IStatsEnhancer... enhancers) {
+ this.enhancers.addAll(Arrays.asList(enhancers));
+ }
+
+ public void addEnhancer(IStatsEnhancer enhancer) {
+ enhancers.add(enhancer);
+ }
+
+ public BasicStats build(Partish p) {
+ BasicStats ret = new BasicStats(p);
+ for (IStatsEnhancer enhancer : enhancers) {
+ ret.apply(enhancer);
+ }
+ return ret;
+ }
+
+ public List<BasicStats> buildAll(HiveConf conf, Collection<Partish> parts) {
+ LOG.info("Number of partishes : " + parts.size());
+
+ final List<BasicStats> ret = new ArrayList<>(parts.size());
+ if (parts.size() <= 1) {
+ for (Partish partish : parts) {
+ ret.add(build(partish));
+ }
+ return ret;
+ }
+
+ List<Future<BasicStats>> futures = new ArrayList<>();
+
+ int threads = conf.getIntVar(ConfVars.METASTORE_FS_HANDLER_THREADS_COUNT);
+
+ final ExecutorService pool;
+ if (threads <= 1) {
+ pool = MoreExecutors.sameThreadExecutor();
+ } else {
+ pool = Executors.newFixedThreadPool(threads, new ThreadFactoryBuilder().setDaemon(true).setNameFormat("Get-Partitions-Size-%d").build());
+ }
+
+ for (final Partish part : parts) {
+ futures.add(pool.submit(new Callable<BasicStats>() {
+ @Override
+ public BasicStats call() throws Exception {
+ return build(part);
+ }
+ }));
+ }
+
+ try {
+ for (int i = 0; i < futures.size(); i++) {
+ ret.add(i, futures.get(i).get());
+ }
+ } catch (InterruptedException | ExecutionException e) {
+ LOG.warn("Exception in processing files ", e);
+ } finally {
+ pool.shutdownNow();
+ }
+ return ret;
+ }
+ }
+
+ public static interface IStatsEnhancer {
+ void apply(BasicStats stats);
+ }
+
+ public static class SetMinRowNumber implements IStatsEnhancer {
+
+ @Override
+ public void apply(BasicStats stats) {
+ if (stats.getNumRows() == 0) {
+ stats.setNumRows(1);
+ }
+ }
+ }
+
+ public static class SetMinRowNumber01 implements IStatsEnhancer {
+
+ @Override
+ public void apply(BasicStats stats) {
+ if (stats.getNumRows() == 0 || stats.getNumRows() == -1) {
+ stats.setNumRows(1);
+ }
+ }
+ }
+
+ public static class RowNumEstimator implements IStatsEnhancer {
+
+ private long avgRowSize;
+
+ public RowNumEstimator(long avgRowSize) {
+ this.avgRowSize = avgRowSize;
+ if (avgRowSize > 0) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Estimated average row size: " + avgRowSize);
+ }
+ }
+ }
+
+ @Override
+ public void apply(BasicStats stats) {
+ // FIXME: there were different logic for part/table; merge these logics later
+ if (stats.partish.getPartition() == null) {
+ if (stats.getNumRows() < 0 && avgRowSize > 0) {
+ stats.setNumRows(stats.getDataSize() / avgRowSize);
+ }
+ } else {
+ if (avgRowSize > 0) {
+ long rc = stats.getNumRows();
+ long s = stats.getDataSize();
+ if (rc <= 0 && s > 0) {
+ rc = s / avgRowSize;
+ stats.setNumRows(rc);
+ }
+
+ if (s <= 0 && rc > 0) {
+ s = StatsUtils.safeMult(rc, avgRowSize);
+ stats.setDataSize(s);
+ }
+ }
+ }
+ if (stats.getNumRows() > 0) {
+ // FIXME: this promotion process should be removed later
+ if (State.PARTIAL.morePreciseThan(stats.state)) {
+ stats.state = State.PARTIAL;
+ }
+ }
+ }
+ }
+
+ public static class DataSizeEstimator implements IStatsEnhancer {
+
+ private HiveConf conf;
+ private float deserFactor;
+
+ public DataSizeEstimator(HiveConf conf) {
+ this.conf = conf;
+ deserFactor = HiveConf.getFloatVar(conf, HiveConf.ConfVars.HIVE_STATS_DESERIALIZATION_FACTOR);
+ }
+
+ @Override
+ public void apply(BasicStats stats) {
+ long ds = stats.getRawDataSize();
+ if (ds <= 0) {
+ ds = stats.getTotalSize();
+
+ // if data size is still 0 then get file size
+ if (ds <= 0) {
+ Path path = stats.partish.getPath();
+ try {
+ ds = getFileSizeForPath(path);
+ } catch (IOException e) {
+ ds = 0L;
+ }
+ }
+ ds = (long) (ds * deserFactor);
+
+ stats.setDataSize(ds);
+ }
+
+ }
+
+ private long getFileSizeForPath(Path path) throws IOException {
+ FileSystem fs = path.getFileSystem(conf);
+ return fs.getContentSummary(path).getLength();
+ }
+
+ }
+
+ private Partish partish;
+
+ private long rowCount;
+ private long totalSize;
+ private long rawDataSize;
+
+ private long currentNumRows;
+ private long currentDataSize;
+ private Statistics.State state;
+
+ public BasicStats(Partish p) {
+ partish = p;
+
+ rowCount = parseLong(StatsSetupConst.ROW_COUNT);
+ rawDataSize = parseLong(StatsSetupConst.RAW_DATA_SIZE);
+ totalSize = parseLong(StatsSetupConst.TOTAL_SIZE);
+
+ currentNumRows = rowCount;
+ currentDataSize = rawDataSize;
+
+ if (currentNumRows > 0) {
+ state = State.COMPLETE;
+ } else {
+ state = State.NONE;
+ }
+ }
+
+
+ public BasicStats(List<BasicStats> partStats) {
+ partish = null;
+ List<Long> nrIn = Lists.newArrayList();
+ List<Long> dsIn = Lists.newArrayList();
+ state = (partStats.size() == 0) ? State.COMPLETE : null;
+ for (BasicStats ps : partStats) {
+ nrIn.add(ps.getNumRows());
+ dsIn.add(ps.getDataSize());
+
+ if (state == null) {
+ state = ps.getState();
+ } else {
+ state = state.merge(ps.getState());
+ }
+ }
+ currentNumRows = StatsUtils.getSumIgnoreNegatives(nrIn);
+ currentDataSize = StatsUtils.getSumIgnoreNegatives(dsIn);
+
+ }
+
+ public long getNumRows() {
+ return currentNumRows;
+ }
+
+ public long getDataSize() {
+ return currentDataSize;
+ }
+
+ public Statistics.State getState() {
+ return state;
+ }
+
+ void apply(IStatsEnhancer estimator) {
+ estimator.apply(this);
+ }
+
+ protected void setNumRows(long l) {
+ currentNumRows = l;
+ }
+
+ protected void setDataSize(long ds) {
+ currentDataSize = ds;
+ }
+
+ protected long getTotalSize() {
+ return totalSize;
+ }
+
+ protected long getRawDataSize() {
+ return rawDataSize;
+ }
+
+ private long parseLong(String fieldName) {
+ Map<String, String> params = partish.getPartParameters();
+ long result = -1;
+
+ if (params != null) {
+ try {
+ result = Long.parseLong(params.get(fieldName));
+ } catch (NumberFormatException e) {
+ result = -1;
+ }
+ }
+ return result;
+ }
+
+ public static BasicStats buildFrom(List<BasicStats> partStats) {
+ return new BasicStats(partStats);
+ }
+
+ @Override
+ public String toString() {
+ return String.format("BasicStats: %d, %d %s", getNumRows(), getDataSize(), getState());
+ }
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/15d7d6dc/ql/src/java/org/apache/hadoop/hive/ql/stats/Partish.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/stats/Partish.java b/ql/src/java/org/apache/hadoop/hive/ql/stats/Partish.java
index 47810e2..54916bd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/stats/Partish.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/stats/Partish.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.stats;
import java.util.Map;
+import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
import org.apache.hadoop.hive.ql.io.AcidUtils;
import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -128,6 +129,11 @@ public abstract class Partish {
public String getSimpleName() {
return String.format("Table %s.%s", table.getDbName(), table.getTableName());
}
+
+ @Override
+ public Path getPath() {
+ return table.getPath();
+ }
}
static class PPart extends Partish {
@@ -185,6 +191,13 @@ public abstract class Partish {
return String.format("Partition %s.%s %s", table.getDbName(), table.getTableName(), partition.getSpec());
}
+ @Override
+ public Path getPath() {
+ return partition.getPartitionPath();
+ }
+
}
+ public abstract Path getPath();
+
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hive/blob/15d7d6dc/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
index e000985..494939a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
@@ -35,7 +35,6 @@ import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.atomic.AtomicInteger;
-import com.google.common.collect.Sets;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -71,6 +70,7 @@ import org.apache.hadoop.hive.ql.plan.ExprNodeFieldDesc;
import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
import org.apache.hadoop.hive.ql.plan.Statistics;
import org.apache.hadoop.hive.ql.plan.Statistics.State;
+import org.apache.hadoop.hive.ql.stats.BasicStats.Factory;
import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge;
import org.apache.hadoop.hive.ql.udf.generic.NDV;
@@ -102,8 +102,8 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableIntObject
import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableLongObjectInspector;
import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableShortObjectInspector;
import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableStringObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableTimestampObjectInspector;
import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableTimestampLocalTZObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableTimestampObjectInspector;
import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo;
import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
@@ -117,6 +117,7 @@ import org.slf4j.LoggerFactory;
import com.google.common.base.Joiner;
import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
import com.google.common.math.LongMath;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
@@ -162,116 +163,47 @@ public class StatsUtils {
fetchColStats, testMode);
}
- private static long getDataSize(HiveConf conf, Table table) {
- long ds = getRawDataSize(table);
- if (ds <= 0) {
- ds = getTotalSize(table);
-
- // if data size is still 0 then get file size
- if (ds <= 0) {
- ds = getFileSizeForTable(conf, table);
- }
- float deserFactor =
- HiveConf.getFloatVar(conf, HiveConf.ConfVars.HIVE_STATS_DESERIALIZATION_FACTOR);
- ds = (long) (ds * deserFactor);
- }
-
- return ds;
- }
-
/**
* Returns number of rows if it exists. Otherwise it estimates number of rows
* based on estimated data size for both partition and non-partitioned table
* RelOptHiveTable's getRowCount uses this.
- *
- * @param conf
- * @param schema
- * @param table
- * @return
*/
- public static long getNumRows(HiveConf conf, List<ColumnInfo> schema, Table table,
- PrunedPartitionList partitionList, AtomicInteger noColsMissingStats) {
+ public static long getNumRows(HiveConf conf, List<ColumnInfo> schema, Table table, PrunedPartitionList partitionList, AtomicInteger noColsMissingStats) {
- boolean shouldEstimateStats = HiveConf.getBoolVar(conf, ConfVars.HIVE_STATS_ESTIMATE_STATS);
-
- if(!table.isPartitioned()) {
- //get actual number of rows from metastore
- long nr = getNumRows(table);
-
- // log warning if row count is missing
- if(nr <= 0) {
- noColsMissingStats.getAndIncrement();
+ List<Partish> inputs = new ArrayList<>();
+ if (table.isPartitioned()) {
+ for (Partition part : partitionList.getNotDeniedPartns()) {
+ inputs.add(Partish.buildFor(table, part));
}
+ } else {
+ inputs.add(Partish.buildFor(table));
+ }
- // if row count exists or stats aren't to be estimated return
- // whatever we have
- if(nr > 0 || !shouldEstimateStats) {
- return nr;
- }
- // go ahead with the estimation
- long ds = getDataSize(conf, table);
- return getNumRows(conf, schema, table, ds);
+ Factory basicStatsFactory = new BasicStats.Factory();
+
+ if (HiveConf.getBoolVar(conf, ConfVars.HIVE_STATS_ESTIMATE_STATS)) {
+ basicStatsFactory.addEnhancer(new BasicStats.DataSizeEstimator(conf));
+ basicStatsFactory.addEnhancer(new BasicStats.RowNumEstimator(estimateRowSizeFromSchema(conf, schema)));
}
- else { // partitioned table
- long nr = 0;
- List<Long> rowCounts = Lists.newArrayList();
- rowCounts = getBasicStatForPartitions(
- table, partitionList.getNotDeniedPartns(), StatsSetupConst.ROW_COUNT);
- nr = getSumIgnoreNegatives(rowCounts);
- // log warning if row count is missing
- if(nr <= 0) {
+ List<BasicStats> results = new ArrayList<>();
+ for (Partish pi : inputs) {
+ BasicStats bStats = new BasicStats(pi);
+ long nr = bStats.getNumRows();
+ // FIXME: this point will be lost after the factory; check that it's really a warning....cleanup/etc
+ if (nr <= 0) {
+ // log warning if row count is missing
noColsMissingStats.getAndIncrement();
}
+ }
- // if row count exists or stats aren't to be estimated return
- // whatever we have
- if(nr > 0 || !shouldEstimateStats) {
- return nr;
- }
-
- // estimate row count
- long ds = 0;
- List<Long> dataSizes = Lists.newArrayList();
-
- dataSizes = getBasicStatForPartitions(
- table, partitionList.getNotDeniedPartns(), StatsSetupConst.RAW_DATA_SIZE);
-
- ds = getSumIgnoreNegatives(dataSizes);
-
- float deserFactor = HiveConf.getFloatVar(conf, HiveConf.ConfVars.HIVE_STATS_DESERIALIZATION_FACTOR);
+ results = basicStatsFactory.buildAll(conf, inputs);
- if (ds <= 0) {
- dataSizes = getBasicStatForPartitions(
- table, partitionList.getNotDeniedPartns(), StatsSetupConst.TOTAL_SIZE);
- dataSizes = safeMult(dataSizes, deserFactor);
- ds = getSumIgnoreNegatives(dataSizes);
- }
-
- // if data size still could not be determined, then fall back to filesytem to get file
- // sizes
- if (ds <= 0 && shouldEstimateStats) {
- dataSizes = getFileSizeForPartitions(conf, partitionList.getNotDeniedPartns());
- dataSizes = safeMult(dataSizes, deserFactor);
- ds = getSumIgnoreNegatives(dataSizes);
- }
+ BasicStats aggregateStat = BasicStats.buildFrom(results);
- int avgRowSize = estimateRowSizeFromSchema(conf, schema);
- if (avgRowSize > 0) {
- setUnknownRcDsToAverage(rowCounts, dataSizes, avgRowSize);
- nr = getSumIgnoreNegatives(rowCounts);
- ds = getSumIgnoreNegatives(dataSizes);
+ aggregateStat.apply(new BasicStats.SetMinRowNumber01());
- // number of rows -1 means that statistics from metastore is not reliable
- if (nr <= 0) {
- nr = ds / avgRowSize;
- }
- }
- if (nr == 0) {
- nr = 1;
- }
- return nr;
- }
+ return aggregateStat.getNumRows();
}
private static void estimateStatsForMissingCols(List<String> neededColumns, List<ColStatistics> columnStats,
@@ -294,26 +226,6 @@ public class StatsUtils {
}
}
- private static long getNumRows(HiveConf conf, List<ColumnInfo> schema, Table table, long ds) {
- long nr = getNumRows(table);
- // number of rows -1 means that statistics from metastore is not reliable
- // and 0 means statistics gathering is disabled
- // estimate only if num rows is -1 since 0 could be actual number of rows
- if (nr < 0) {
- int avgRowSize = estimateRowSizeFromSchema(conf, schema);
- if (avgRowSize > 0) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Estimated average row size: " + avgRowSize);
- }
- nr = ds / avgRowSize;
- }
- }
- if(nr == 0 || nr == -1) {
- return 1;
- }
- return nr;
- }
-
public static Statistics collectStatistics(HiveConf conf, PrunedPartitionList partList,
Table table, List<ColumnInfo> schema, List<String> neededColumns, ColumnStatsList colStatsCache,
List<String> referencedColumns, boolean fetchColStats)
@@ -322,24 +234,33 @@ public class StatsUtils {
referencedColumns, fetchColStats, false);
}
- private static Statistics collectStatistics(HiveConf conf, PrunedPartitionList partList,
- Table table, List<ColumnInfo> schema, List<String> neededColumns, ColumnStatsList colStatsCache,
- List<String> referencedColumns, boolean fetchColStats, boolean failIfCacheMiss)
- throws HiveException {
+ private static Statistics collectStatistics(HiveConf conf, PrunedPartitionList partList, Table table,
+ List<ColumnInfo> schema, List<String> neededColumns, ColumnStatsList colStatsCache,
+ List<String> referencedColumns, boolean fetchColStats, boolean failIfCacheMiss) throws HiveException {
Statistics stats = null;
- float deserFactor =
- HiveConf.getFloatVar(conf, HiveConf.ConfVars.HIVE_STATS_DESERIALIZATION_FACTOR);
boolean shouldEstimateStats = HiveConf.getBoolVar(conf, ConfVars.HIVE_STATS_ESTIMATE_STATS);
if (!table.isPartitioned()) {
- //getDataSize tries to estimate stats if it doesn't exist using file size
- // we would like to avoid file system calls if it too expensive
- long ds = shouldEstimateStats? getDataSize(conf, table): getRawDataSize(table);
- long nr = getNumRows(conf, schema, table, ds);
+ Factory basicStatsFactory = new BasicStats.Factory();
+
+ if (shouldEstimateStats) {
+ basicStatsFactory.addEnhancer(new BasicStats.DataSizeEstimator(conf));
+ }
+
+ // long ds = shouldEstimateStats? getDataSize(conf, table): getRawDataSize(table);
+ basicStatsFactory.addEnhancer(new BasicStats.RowNumEstimator(estimateRowSizeFromSchema(conf, schema)));
+ basicStatsFactory.addEnhancer(new BasicStats.SetMinRowNumber01());
+
+ BasicStats basicStats = basicStatsFactory.build(Partish.buildFor(table));
+
+ // long nr = getNumRows(conf, schema, neededColumns, table, ds);
+ long ds = basicStats.getDataSize();
+ long nr = basicStats.getNumRows();
List<ColStatistics> colStats = Lists.newArrayList();
+
if (fetchColStats) {
colStats = getTableColumnStats(table, schema, neededColumns, colStatsCache);
if(colStats == null) {
@@ -359,59 +280,48 @@ public class StatsUtils {
stats.setColumnStatsState(deriveStatType(colStats, neededColumns));
stats.addToColumnStats(colStats);
} else if (partList != null) {
+
// For partitioned tables, get the size of all the partitions after pruning
// the partitions that are not required
- long nr = 0;
- long ds = 0;
- List<Long> rowCounts = Lists.newArrayList();
- List<Long> dataSizes = Lists.newArrayList();
+ Factory basicStatsFactory = new Factory();
+ if (shouldEstimateStats) {
+ // FIXME: misses parallel
+ basicStatsFactory.addEnhancer(new BasicStats.DataSizeEstimator(conf));
+ }
- rowCounts = getBasicStatForPartitions(table, partList.getNotDeniedPartns(), StatsSetupConst.ROW_COUNT);
- dataSizes = getBasicStatForPartitions(table, partList.getNotDeniedPartns(), StatsSetupConst.RAW_DATA_SIZE);
+ basicStatsFactory.addEnhancer(new BasicStats.RowNumEstimator(estimateRowSizeFromSchema(conf, schema)));
- nr = getSumIgnoreNegatives(rowCounts);
- ds = getSumIgnoreNegatives(dataSizes);
- if (ds <= 0) {
- dataSizes = getBasicStatForPartitions(table, partList.getNotDeniedPartns(), StatsSetupConst.TOTAL_SIZE);
- dataSizes = safeMult(dataSizes, deserFactor);
- ds = getSumIgnoreNegatives(dataSizes);
- }
+ List<BasicStats> partStats = new ArrayList<>();
- // if data size still could not be determined, then fall back to filesytem to get file
- // sizes
- if (ds <= 0 && shouldEstimateStats) {
- dataSizes = getFileSizeForPartitions(conf, partList.getNotDeniedPartns());
- dataSizes = safeMult(dataSizes, deserFactor);
- ds = getSumIgnoreNegatives(dataSizes);
+ for (Partition p : partList.getNotDeniedPartns()) {
+ BasicStats basicStats = basicStatsFactory.build(Partish.buildFor(table, p));
+ partStats.add(basicStats);
}
+ BasicStats bbs = BasicStats.buildFrom(partStats);
- int avgRowSize = estimateRowSizeFromSchema(conf, schema);
- if (avgRowSize > 0) {
- setUnknownRcDsToAverage(rowCounts, dataSizes, avgRowSize);
- nr = getSumIgnoreNegatives(rowCounts);
- ds = getSumIgnoreNegatives(dataSizes);
-
- // number of rows -1 means that statistics from metastore is not reliable
- if (nr <= 0) {
- nr = ds / avgRowSize;
- }
+ List<Long> rowCounts = Lists.newArrayList();
+ for (BasicStats basicStats : partStats) {
+ rowCounts.add(basicStats.getNumRows());
}
- // Minimum values
+ long nr = bbs.getNumRows();
+ long ds = bbs.getDataSize();
+
if (nr == 0) {
- nr = 1;
+ nr=1;
}
stats = new Statistics(nr, ds);
-
- // if at least a partition does not contain row count then mark basic stats state as PARTIAL
- if (containsNonPositives(rowCounts) &&
- stats.getBasicStatsState().equals(State.COMPLETE)) {
- stats.setBasicStatsState(State.PARTIAL);
+ stats.setBasicStatsState(bbs.getState());
+ if (nr > 0) {
+ // FIXME: this promotion process should be removed later
+ if (State.PARTIAL.morePreciseThan(bbs.getState())) {
+ stats.setBasicStatsState(State.PARTIAL);
+ }
}
+
if (fetchColStats) {
- List<String> partitionCols = getPartitionColumns(
- schema, neededColumns, referencedColumns);
+ List<String> partitionCols = getPartitionColumns(schema, neededColumns, referencedColumns);
// We will retrieve stats from the metastore only for columns that are not cached
List<String> neededColsToRetrieve;
@@ -746,26 +656,6 @@ public class StatsUtils {
return range;
}
- private static void setUnknownRcDsToAverage(
- List<Long> rowCounts, List<Long> dataSizes, int avgRowSize) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Estimated average row size: " + avgRowSize);
- }
- for (int i = 0; i < rowCounts.size(); i++) {
- long rc = rowCounts.get(i);
- long s = dataSizes.get(i);
- if (rc <= 0 && s > 0) {
- rc = s / avgRowSize;
- rowCounts.set(i, rc);
- }
-
- if (s <= 0 && rc > 0) {
- s = safeMult(rc, avgRowSize);
- dataSizes.set(i, s);
- }
- }
- }
-
public static int estimateRowSizeFromSchema(HiveConf conf, List<ColumnInfo> schema) {
List<String> neededColumns = new ArrayList<>();
for (ColumnInfo ci : schema) {
@@ -838,6 +728,7 @@ public class StatsUtils {
* - partition list
* @return sizes of partitions
*/
+ @Deprecated
public static List<Long> getFileSizeForPartitions(final HiveConf conf, List<Partition> parts) {
LOG.info("Number of partitions : " + parts.size());
ArrayList<Future<Long>> futures = new ArrayList<>();
@@ -878,7 +769,7 @@ public class StatsUtils {
return sizes;
}
- private static boolean containsNonPositives(List<Long> vals) {
+ public static boolean containsNonPositives(List<Long> vals) {
for (Long val : vals) {
if (val <= 0L) {
return true;
@@ -1737,12 +1628,13 @@ public class StatsUtils {
}
long countDistincts = ndvs.isEmpty() ? numRows : addWithExpDecay(ndvs);
return Collections.min(Lists.newArrayList(countDistincts, udfNDV, numRows));
- }
+ }
/**
* Get number of rows of a give table
* @return number of rows
*/
+ @Deprecated
public static long getNumRows(Table table) {
return getBasicStatForTable(table, StatsSetupConst.ROW_COUNT);
}
@@ -1771,6 +1663,7 @@ public class StatsUtils {
* - type of stats
* @return value of stats
*/
+ @Deprecated
public static long getBasicStatForTable(Table table, String statType) {
Map<String, String> params = table.getParameters();
long result = -1;
http://git-wip-us.apache.org/repos/asf/hive/blob/15d7d6dc/ql/src/test/org/apache/hadoop/hive/ql/stats/TestBasicStats.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/stats/TestBasicStats.java b/ql/src/test/org/apache/hadoop/hive/ql/stats/TestBasicStats.java
new file mode 100644
index 0000000..eb362f7
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/stats/TestBasicStats.java
@@ -0,0 +1,123 @@
+/**
+ * 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 static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.doReturn;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.ql.metadata.Partition;
+import org.apache.hadoop.hive.ql.plan.Statistics.State;
+import org.apache.hadoop.hive.ql.stats.BasicStats;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import com.google.common.collect.Lists;
+
+public class TestBasicStats {
+
+ public static class LocalPartishBuilder {
+ Map<String, String> params = new HashMap<>();
+
+ public LocalPartishBuilder numRows(int i) {
+ params.put(StatsSetupConst.ROW_COUNT, String.valueOf(i));
+ return this;
+ }
+
+ public LocalPartishBuilder rawDataSize(int i) {
+ params.put(StatsSetupConst.RAW_DATA_SIZE, String.valueOf(i));
+ return this;
+ }
+
+ public LocalPartishBuilder totalSize(int i) {
+ params.put(StatsSetupConst.TOTAL_SIZE, String.valueOf(i));
+ return this;
+ }
+
+ public Partish buildPartition() {
+ Partition partition = Mockito.mock(Partition.class);
+ org.apache.hadoop.hive.metastore.api.Partition tpartition = Mockito.mock(org.apache.hadoop.hive.metastore.api.Partition.class);
+ doReturn(tpartition).when(partition).getTPartition();
+ doReturn(params).when(tpartition).getParameters();
+ return Partish.buildFor(null, partition);
+ }
+ }
+
+ @Test
+ public void testDataSizeEstimator() {
+ Partish p1 = new LocalPartishBuilder().totalSize(10).buildPartition();
+
+ HiveConf conf = new HiveConf();
+ conf.setFloatVar(ConfVars.HIVE_STATS_DESERIALIZATION_FACTOR, 13.0f);
+ BasicStats.Factory factory = new BasicStats.Factory(new BasicStats.DataSizeEstimator(conf));
+
+ BasicStats res = factory.build(p1);
+
+ assertEquals(130, res.getDataSize());
+ }
+
+ @Test
+ public void mergeWithEmpty() {
+
+ HiveConf conf = new HiveConf();
+ int avgRowSize = 100;
+ int r0 = 13;
+ int r1 = 15;
+ int deserFactor = (int) conf.getFloatVar(ConfVars.HIVE_STATS_DESERIALIZATION_FACTOR);
+ Partish p0 = new LocalPartishBuilder().numRows(r0).rawDataSize(avgRowSize * r0).buildPartition();
+ Partish p1 = new LocalPartishBuilder().totalSize(r1 * avgRowSize / deserFactor).buildPartition();
+
+ BasicStats.Factory factory =
+ new BasicStats.Factory(new BasicStats.DataSizeEstimator(conf), new BasicStats.RowNumEstimator(avgRowSize));
+
+ BasicStats bs0 = factory.build(p0);
+ BasicStats bs1 = factory.build(p1);
+
+ BasicStats res = BasicStats.buildFrom(Lists.newArrayList(bs0, bs1));
+
+ assertEquals(r0 + r1, res.getNumRows());
+ assertEquals(avgRowSize * (r0 + r1), res.getDataSize());
+ }
+
+ @Test
+ @Ignore("HIVE-18062 will fix this")
+ public void mergedKeepsPartialStateEvenIfValuesAreSuccessfullyEstimated() {
+ Partish p0 = new LocalPartishBuilder().numRows(10).rawDataSize(100).buildPartition();
+ Partish p1 = new LocalPartishBuilder().totalSize(10).buildPartition();
+
+ HiveConf conf = new HiveConf();
+ BasicStats.Factory factory =
+ new BasicStats.Factory(new BasicStats.DataSizeEstimator(conf), new BasicStats.RowNumEstimator(10));
+
+ BasicStats bs0 = factory.build(p0);
+ BasicStats bs1 = factory.build(p1);
+
+ BasicStats res = BasicStats.buildFrom(Lists.newArrayList(bs0, bs1));
+
+ assertEquals(State.PARTIAL, res.getState());
+ }
+
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/15d7d6dc/ql/src/test/queries/clientpositive/stats8.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/stats8.q b/ql/src/test/queries/clientpositive/stats8.q
index e0a71f7..98e9194 100644
--- a/ql/src/test/queries/clientpositive/stats8.q
+++ b/ql/src/test/queries/clientpositive/stats8.q
@@ -7,11 +7,13 @@ set hive.exec.dynamic.partition.mode=nonstrict;
create table analyze_srcpart_n1 like srcpart;
insert overwrite table analyze_srcpart_n1 partition (ds, hr) select * from srcpart where ds is not null;
+describe formatted analyze_srcpart_n1 PARTITION(ds='2008-04-08',hr=11);
explain analyze table analyze_srcpart_n1 PARTITION(ds='2008-04-08',hr=11) compute statistics;
analyze table analyze_srcpart_n1 PARTITION(ds='2008-04-08',hr=11) compute statistics;
describe formatted analyze_srcpart_n1 PARTITION(ds='2008-04-08',hr=11);
describe formatted analyze_srcpart_n1;
+describe formatted analyze_srcpart_n1 PARTITION(ds='2008-04-08',hr=12);
explain analyze table analyze_srcpart_n1 PARTITION(ds='2008-04-08',hr=12) compute statistics;
analyze table analyze_srcpart_n1 PARTITION(ds='2008-04-08',hr=12) compute statistics;
describe formatted analyze_srcpart_n1 PARTITION(ds='2008-04-08',hr=12);
http://git-wip-us.apache.org/repos/asf/hive/blob/15d7d6dc/ql/src/test/results/clientnegative/bucket_mapjoin_mismatch1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/bucket_mapjoin_mismatch1.q.out b/ql/src/test/results/clientnegative/bucket_mapjoin_mismatch1.q.out
index 13028d7..7b32317 100644
--- a/ql/src/test/results/clientnegative/bucket_mapjoin_mismatch1.q.out
+++ b/ql/src/test/results/clientnegative/bucket_mapjoin_mismatch1.q.out
@@ -95,36 +95,36 @@ STAGE PLANS:
TableScan
alias: a
filterExpr: ((ds = '2008-04-08') and key is not null) (type: boolean)
- Statistics: Num rows: 108 Data size: 42000 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 108 Data size: 42000 Basic stats: PARTIAL Column stats: NONE
Filter Operator
predicate: key is not null (type: boolean)
- Statistics: Num rows: 108 Data size: 42000 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 108 Data size: 42000 Basic stats: PARTIAL Column stats: NONE
Select Operator
expressions: key (type: int), value (type: string)
outputColumnNames: _col0, _col1
- Statistics: Num rows: 108 Data size: 42000 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 108 Data size: 42000 Basic stats: PARTIAL Column stats: NONE
Reduce Output Operator
key expressions: _col0 (type: int)
sort order: +
Map-reduce partition columns: _col0 (type: int)
- Statistics: Num rows: 108 Data size: 42000 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 108 Data size: 42000 Basic stats: PARTIAL Column stats: NONE
value expressions: _col1 (type: string)
TableScan
alias: b
filterExpr: ((ds = '2008-04-08') and key is not null) (type: boolean)
- Statistics: Num rows: 70 Data size: 27500 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 70 Data size: 27500 Basic stats: PARTIAL Column stats: NONE
Filter Operator
predicate: key is not null (type: boolean)
- Statistics: Num rows: 70 Data size: 27500 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 70 Data size: 27500 Basic stats: PARTIAL Column stats: NONE
Select Operator
expressions: key (type: int), value (type: string)
outputColumnNames: _col0, _col1
- Statistics: Num rows: 70 Data size: 27500 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 70 Data size: 27500 Basic stats: PARTIAL Column stats: NONE
Reduce Output Operator
key expressions: _col0 (type: int)
sort order: +
Map-reduce partition columns: _col0 (type: int)
- Statistics: Num rows: 70 Data size: 27500 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 70 Data size: 27500 Basic stats: PARTIAL Column stats: NONE
value expressions: _col1 (type: string)
Reduce Operator Tree:
Join Operator
@@ -134,14 +134,14 @@ STAGE PLANS:
0 _col0 (type: int)
1 _col0 (type: int)
outputColumnNames: _col0, _col1, _col4
- Statistics: Num rows: 118 Data size: 46200 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 118 Data size: 46200 Basic stats: PARTIAL Column stats: NONE
Select Operator
expressions: _col0 (type: int), _col1 (type: string), _col4 (type: string)
outputColumnNames: _col0, _col1, _col2
- Statistics: Num rows: 118 Data size: 46200 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 118 Data size: 46200 Basic stats: PARTIAL Column stats: NONE
File Output Operator
compressed: false
- Statistics: Num rows: 118 Data size: 46200 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 118 Data size: 46200 Basic stats: PARTIAL Column stats: NONE
table:
input format: org.apache.hadoop.mapred.SequenceFileInputFormat
output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -180,10 +180,10 @@ STAGE PLANS:
TableScan
alias: b
filterExpr: ((ds = '2008-04-08') and key is not null) (type: boolean)
- Statistics: Num rows: 92 Data size: 27500 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 92 Data size: 27500 Basic stats: PARTIAL Column stats: NONE
Filter Operator
predicate: key is not null (type: boolean)
- Statistics: Num rows: 92 Data size: 27500 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 92 Data size: 27500 Basic stats: PARTIAL Column stats: NONE
HashTable Sink Operator
keys:
0 key (type: int)
@@ -195,10 +195,10 @@ STAGE PLANS:
TableScan
alias: a
filterExpr: ((ds = '2008-04-08') and key is not null) (type: boolean)
- Statistics: Num rows: 140 Data size: 42000 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 140 Data size: 42000 Basic stats: PARTIAL Column stats: NONE
Filter Operator
predicate: key is not null (type: boolean)
- Statistics: Num rows: 140 Data size: 42000 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 140 Data size: 42000 Basic stats: PARTIAL Column stats: NONE
Map Join Operator
condition map:
Inner Join 0 to 1
@@ -206,14 +206,14 @@ STAGE PLANS:
0 key (type: int)
1 key (type: int)
outputColumnNames: _col0, _col1, _col7
- Statistics: Num rows: 154 Data size: 46200 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 154 Data size: 46200 Basic stats: PARTIAL Column stats: NONE
Select Operator
expressions: _col0 (type: int), _col1 (type: string), _col7 (type: string)
outputColumnNames: _col0, _col1, _col2
- Statistics: Num rows: 154 Data size: 46200 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 154 Data size: 46200 Basic stats: PARTIAL Column stats: NONE
File Output Operator
compressed: false
- Statistics: Num rows: 154 Data size: 46200 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 154 Data size: 46200 Basic stats: PARTIAL Column stats: NONE
table:
input format: org.apache.hadoop.mapred.SequenceFileInputFormat
output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
http://git-wip-us.apache.org/repos/asf/hive/blob/15d7d6dc/ql/src/test/results/clientpositive/acid_table_stats.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/acid_table_stats.q.out b/ql/src/test/results/clientpositive/acid_table_stats.q.out
index acda186..084d232 100644
--- a/ql/src/test/results/clientpositive/acid_table_stats.q.out
+++ b/ql/src/test/results/clientpositive/acid_table_stats.q.out
@@ -133,17 +133,17 @@ STAGE PLANS:
TableScan
alias: acid
filterExpr: (ds = '2008-04-08') (type: boolean)
- Statistics: Num rows: 83 Data size: 40630 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 83 Data size: 40630 Basic stats: PARTIAL Column stats: NONE
Select Operator
- Statistics: Num rows: 83 Data size: 40630 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 83 Data size: 40630 Basic stats: PARTIAL Column stats: NONE
Group By Operator
aggregations: count()
mode: hash
outputColumnNames: _col0
- Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
Reduce Output Operator
sort order:
- Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
value expressions: _col0 (type: bigint)
Execution mode: vectorized
Reduce Operator Tree:
@@ -151,10 +151,10 @@ STAGE PLANS:
aggregations: count(VALUE._col0)
mode: mergepartial
outputColumnNames: _col0
- Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
File Output Operator
compressed: false
- Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
table:
input format: org.apache.hadoop.mapred.SequenceFileInputFormat
output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
http://git-wip-us.apache.org/repos/asf/hive/blob/15d7d6dc/ql/src/test/results/clientpositive/analyze_table_null_partition.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/analyze_table_null_partition.q.out b/ql/src/test/results/clientpositive/analyze_table_null_partition.q.out
index 3e477af..a8cfb97 100644
--- a/ql/src/test/results/clientpositive/analyze_table_null_partition.q.out
+++ b/ql/src/test/results/clientpositive/analyze_table_null_partition.q.out
@@ -282,12 +282,12 @@ STAGE PLANS:
Processor Tree:
TableScan
alias: test2_n6
- Statistics: Num rows: 5 Data size: 299 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 5 Data size: 21 Basic stats: COMPLETE Column stats: NONE
GatherStats: false
Select Operator
expressions: name (type: string), age (type: int)
outputColumnNames: _col0, _col1
- Statistics: Num rows: 5 Data size: 299 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 5 Data size: 21 Basic stats: COMPLETE Column stats: NONE
ListSink
PREHOOK: query: DROP TABLE test1_n9
http://git-wip-us.apache.org/repos/asf/hive/blob/15d7d6dc/ql/src/test/results/clientpositive/annotate_stats_part.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/annotate_stats_part.q.out b/ql/src/test/results/clientpositive/annotate_stats_part.q.out
index 2bd4d17..29ef214 100644
--- a/ql/src/test/results/clientpositive/annotate_stats_part.q.out
+++ b/ql/src/test/results/clientpositive/annotate_stats_part.q.out
@@ -90,11 +90,11 @@ STAGE PLANS:
Processor Tree:
TableScan
alias: loc_orc_n4
- Statistics: Num rows: 20 Data size: 15680 Basic stats: COMPLETE Column stats: PARTIAL
+ Statistics: Num rows: 20 Data size: 15680 Basic stats: PARTIAL Column stats: PARTIAL
Select Operator
expressions: state (type: string), locid (type: int), zip (type: bigint), year (type: string)
outputColumnNames: _col0, _col1, _col2, _col3
- Statistics: Num rows: 20 Data size: 7600 Basic stats: COMPLETE Column stats: PARTIAL
+ Statistics: Num rows: 20 Data size: 15680 Basic stats: PARTIAL Column stats: PARTIAL
ListSink
PREHOOK: query: analyze table loc_orc_n4 partition(year='2001') compute statistics
@@ -122,11 +122,11 @@ STAGE PLANS:
TableScan
alias: loc_orc_n4
filterExpr: (year = '__HIVE_DEFAULT_PARTITION__') (type: boolean)
- Statistics: Num rows: 9 Data size: 5364 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 9 Data size: 5364 Basic stats: PARTIAL Column stats: NONE
Select Operator
expressions: state (type: string), locid (type: int), zip (type: bigint), '__HIVE_DEFAULT_PARTITION__' (type: string)
outputColumnNames: _col0, _col1, _col2, _col3
- Statistics: Num rows: 9 Data size: 5364 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 9 Data size: 5364 Basic stats: PARTIAL Column stats: NONE
ListSink
PREHOOK: query: explain select * from loc_orc_n4
@@ -143,11 +143,11 @@ STAGE PLANS:
Processor Tree:
TableScan
alias: loc_orc_n4
- Statistics: Num rows: 7 Data size: 3338 Basic stats: PARTIAL Column stats: PARTIAL
+ Statistics: Num rows: 16 Data size: 10358 Basic stats: PARTIAL Column stats: PARTIAL
Select Operator
expressions: state (type: string), locid (type: int), zip (type: bigint), year (type: string)
outputColumnNames: _col0, _col1, _col2, _col3
- Statistics: Num rows: 7 Data size: 3338 Basic stats: PARTIAL Column stats: PARTIAL
+ Statistics: Num rows: 16 Data size: 10358 Basic stats: PARTIAL Column stats: PARTIAL
ListSink
PREHOOK: query: explain select * from loc_orc_n4 where year='2001'
http://git-wip-us.apache.org/repos/asf/hive/blob/15d7d6dc/ql/src/test/results/clientpositive/autoColumnStats_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/autoColumnStats_2.q.out b/ql/src/test/results/clientpositive/autoColumnStats_2.q.out
index 8e76396..07f1cba 100644
--- a/ql/src/test/results/clientpositive/autoColumnStats_2.q.out
+++ b/ql/src/test/results/clientpositive/autoColumnStats_2.q.out
@@ -785,11 +785,11 @@ STAGE PLANS:
TableScan
alias: alter5
filterExpr: (dt = 'a') (type: boolean)
- Statistics: Num rows: 49 Data size: 4263 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 49 Data size: 4263 Basic stats: PARTIAL Column stats: COMPLETE
Select Operator
expressions: col1 (type: string), 'a' (type: string)
outputColumnNames: _col0, _col1
- Statistics: Num rows: 49 Data size: 8428 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 49 Data size: 4263 Basic stats: PARTIAL Column stats: COMPLETE
ListSink
PREHOOK: query: drop table src_stat_part
http://git-wip-us.apache.org/repos/asf/hive/blob/15d7d6dc/ql/src/test/results/clientpositive/beeline/smb_mapjoin_10.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/beeline/smb_mapjoin_10.q.out b/ql/src/test/results/clientpositive/beeline/smb_mapjoin_10.q.out
index 6938b0e..8b48c7f 100644
--- a/ql/src/test/results/clientpositive/beeline/smb_mapjoin_10.q.out
+++ b/ql/src/test/results/clientpositive/beeline/smb_mapjoin_10.q.out
@@ -79,10 +79,10 @@ STAGE PLANS:
TableScan
alias: b
filterExpr: (userid is not null and pageid is not null and postid is not null and type is not null) (type: boolean)
- Statistics: Num rows: 13 Data size: 4140 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 13 Data size: 4140 Basic stats: PARTIAL Column stats: NONE
Filter Operator
predicate: (pageid is not null and postid is not null and type is not null and userid is not null) (type: boolean)
- Statistics: Num rows: 13 Data size: 4140 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 13 Data size: 4140 Basic stats: PARTIAL Column stats: NONE
Sorted Merge Bucket Map Join Operator
condition map:
Inner Join 0 to 1
http://git-wip-us.apache.org/repos/asf/hive/blob/15d7d6dc/ql/src/test/results/clientpositive/bucket_map_join_spark1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/bucket_map_join_spark1.q.out b/ql/src/test/results/clientpositive/bucket_map_join_spark1.q.out
index 4650034..9fc4fd9 100644
--- a/ql/src/test/results/clientpositive/bucket_map_join_spark1.q.out
+++ b/ql/src/test/results/clientpositive/bucket_map_join_spark1.q.out
@@ -184,16 +184,16 @@ STAGE PLANS:
TableScan
alias: a
filterExpr: key is not null (type: boolean)
- Statistics: Num rows: 149 Data size: 58120 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 149 Data size: 58120 Basic stats: PARTIAL Column stats: NONE
GatherStats: false
Filter Operator
isSamplingPred: false
predicate: key is not null (type: boolean)
- Statistics: Num rows: 149 Data size: 58120 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 149 Data size: 58120 Basic stats: PARTIAL Column stats: NONE
Select Operator
expressions: key (type: int), value (type: string)
outputColumnNames: _col0, _col1
- Statistics: Num rows: 149 Data size: 58120 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 149 Data size: 58120 Basic stats: PARTIAL Column stats: NONE
HashTable Sink Operator
keys:
0 _col0 (type: int)
@@ -206,16 +206,16 @@ STAGE PLANS:
TableScan
alias: b
filterExpr: ((ds = '2008-04-08') and key is not null) (type: boolean)
- Statistics: Num rows: 149 Data size: 58120 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 149 Data size: 58120 Basic stats: PARTIAL Column stats: NONE
GatherStats: false
Filter Operator
isSamplingPred: false
predicate: key is not null (type: boolean)
- Statistics: Num rows: 149 Data size: 58120 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 149 Data size: 58120 Basic stats: PARTIAL Column stats: NONE
Select Operator
expressions: key (type: int), value (type: string)
outputColumnNames: _col0, _col1
- Statistics: Num rows: 149 Data size: 58120 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 149 Data size: 58120 Basic stats: PARTIAL Column stats: NONE
Map Join Operator
condition map:
Inner Join 0 to 1
@@ -224,17 +224,17 @@ STAGE PLANS:
1 _col0 (type: int)
outputColumnNames: _col0, _col1, _col3
Position of Big Table: 1
- Statistics: Num rows: 163 Data size: 63932 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 163 Data size: 63932 Basic stats: PARTIAL Column stats: NONE
Select Operator
expressions: UDFToString(_col0) (type: string), _col1 (type: string), _col3 (type: string)
outputColumnNames: _col0, _col1, _col2
- Statistics: Num rows: 163 Data size: 63932 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 163 Data size: 63932 Basic stats: PARTIAL Column stats: NONE
File Output Operator
compressed: false
GlobalTableId: 1
#### A masked pattern was here ####
NumFilesPerFileSink: 1
- Statistics: Num rows: 163 Data size: 63932 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 163 Data size: 63932 Basic stats: PARTIAL Column stats: NONE
#### A masked pattern was here ####
table:
input format: org.apache.hadoop.mapred.TextInputFormat
@@ -265,12 +265,12 @@ STAGE PLANS:
Select Operator
expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
outputColumnNames: key, value1, value2
- Statistics: Num rows: 163 Data size: 63932 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 163 Data size: 63932 Basic stats: PARTIAL Column stats: NONE
Group By Operator
aggregations: compute_stats(key, 'hll'), compute_stats(value1, 'hll'), compute_stats(value2, 'hll')
mode: hash
outputColumnNames: _col0, _col1, _col2
- Statistics: Num rows: 1 Data size: 1320 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
File Output Operator
compressed: false
GlobalTableId: 0
@@ -442,7 +442,7 @@ STAGE PLANS:
Reduce Output Operator
null sort order:
sort order:
- Statistics: Num rows: 1 Data size: 1320 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
tag: -1
value expressions: _col0 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col1 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col2 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>)
auto parallelism: false
@@ -479,13 +479,13 @@ STAGE PLANS:
aggregations: compute_stats(VALUE._col0), compute_stats(VALUE._col1), compute_stats(VALUE._col2)
mode: mergepartial
outputColumnNames: _col0, _col1, _col2
- Statistics: Num rows: 1 Data size: 1320 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
File Output Operator
compressed: false
GlobalTableId: 0
#### A masked pattern was here ####
NumFilesPerFileSink: 1
- Statistics: Num rows: 1 Data size: 1320 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
#### A masked pattern was here ####
table:
input format: org.apache.hadoop.mapred.SequenceFileInputFormat
@@ -628,16 +628,16 @@ STAGE PLANS:
TableScan
alias: a
filterExpr: key is not null (type: boolean)
- Statistics: Num rows: 149 Data size: 58120 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 149 Data size: 58120 Basic stats: PARTIAL Column stats: NONE
GatherStats: false
Filter Operator
isSamplingPred: false
predicate: key is not null (type: boolean)
- Statistics: Num rows: 149 Data size: 58120 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 149 Data size: 58120 Basic stats: PARTIAL Column stats: NONE
Select Operator
expressions: key (type: int), value (type: string)
outputColumnNames: _col0, _col1
- Statistics: Num rows: 149 Data size: 58120 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 149 Data size: 58120 Basic stats: PARTIAL Column stats: NONE
HashTable Sink Operator
keys:
0 _col0 (type: int)
@@ -650,16 +650,16 @@ STAGE PLANS:
TableScan
alias: b
filterExpr: ((ds = '2008-04-08') and key is not null) (type: boolean)
- Statistics: Num rows: 149 Data size: 58120 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 149 Data size: 58120 Basic stats: PARTIAL Column stats: NONE
GatherStats: false
Filter Operator
isSamplingPred: false
predicate: key is not null (type: boolean)
- Statistics: Num rows: 149 Data size: 58120 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 149 Data size: 58120 Basic stats: PARTIAL Column stats: NONE
Select Operator
expressions: key (type: int), value (type: string)
outputColumnNames: _col0, _col1
- Statistics: Num rows: 149 Data size: 58120 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 149 Data size: 58120 Basic stats: PARTIAL Column stats: NONE
Map Join Operator
condition map:
Inner Join 0 to 1
@@ -668,17 +668,17 @@ STAGE PLANS:
1 _col0 (type: int)
outputColumnNames: _col0, _col1, _col3
Position of Big Table: 1
- Statistics: Num rows: 163 Data size: 63932 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 163 Data size: 63932 Basic stats: PARTIAL Column stats: NONE
Select Operator
expressions: UDFToString(_col0) (type: string), _col1 (type: string), _col3 (type: string)
outputColumnNames: _col0, _col1, _col2
- Statistics: Num rows: 163 Data size: 63932 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 163 Data size: 63932 Basic stats: PARTIAL Column stats: NONE
File Output Operator
compressed: false
GlobalTableId: 1
#### A masked pattern was here ####
NumFilesPerFileSink: 1
- Statistics: Num rows: 163 Data size: 63932 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 163 Data size: 63932 Basic stats: PARTIAL Column stats: NONE
#### A masked pattern was here ####
table:
input format: org.apache.hadoop.mapred.TextInputFormat
@@ -709,12 +709,12 @@ STAGE PLANS:
Select Operator
expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
outputColumnNames: key, value1, value2
- Statistics: Num rows: 163 Data size: 63932 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 163 Data size: 63932 Basic stats: PARTIAL Column stats: NONE
Group By Operator
aggregations: compute_stats(key, 'hll'), compute_stats(value1, 'hll'), compute_stats(value2, 'hll')
mode: hash
outputColumnNames: _col0, _col1, _col2
- Statistics: Num rows: 1 Data size: 1320 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
File Output Operator
compressed: false
GlobalTableId: 0
@@ -886,7 +886,7 @@ STAGE PLANS:
Reduce Output Operator
null sort order:
sort order:
- Statistics: Num rows: 1 Data size: 1320 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
tag: -1
value expressions: _col0 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col1 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col2 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>)
auto parallelism: false
@@ -923,13 +923,13 @@ STAGE PLANS:
aggregations: compute_stats(VALUE._col0), compute_stats(VALUE._col1), compute_stats(VALUE._col2)
mode: mergepartial
outputColumnNames: _col0, _col1, _col2
- Statistics: Num rows: 1 Data size: 1320 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
File Output Operator
compressed: false
GlobalTableId: 0
#### A masked pattern was here ####
NumFilesPerFileSink: 1
- Statistics: Num rows: 1 Data size: 1320 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
#### A masked pattern was here ####
table:
input format: org.apache.hadoop.mapred.SequenceFileInputFormat
http://git-wip-us.apache.org/repos/asf/hive/blob/15d7d6dc/ql/src/test/results/clientpositive/bucket_map_join_spark2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/bucket_map_join_spark2.q.out b/ql/src/test/results/clientpositive/bucket_map_join_spark2.q.out
index 64c5bf4..61eea99 100644
--- a/ql/src/test/results/clientpositive/bucket_map_join_spark2.q.out
+++ b/ql/src/test/results/clientpositive/bucket_map_join_spark2.q.out
@@ -168,16 +168,16 @@ STAGE PLANS:
TableScan
alias: b
filterExpr: ((ds = '2008-04-08') and key is not null) (type: boolean)
- Statistics: Num rows: 78 Data size: 30620 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 78 Data size: 30620 Basic stats: PARTIAL Column stats: NONE
GatherStats: false
Filter Operator
isSamplingPred: false
predicate: key is not null (type: boolean)
- Statistics: Num rows: 78 Data size: 30620 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 78 Data size: 30620 Basic stats: PARTIAL Column stats: NONE
Select Operator
expressions: key (type: int), value (type: string)
outputColumnNames: _col0, _col1
- Statistics: Num rows: 78 Data size: 30620 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 78 Data size: 30620 Basic stats: PARTIAL Column stats: NONE
HashTable Sink Operator
keys:
0 _col0 (type: int)
@@ -190,16 +190,16 @@ STAGE PLANS:
TableScan
alias: a
filterExpr: key is not null (type: boolean)
- Statistics: Num rows: 149 Data size: 58120 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 149 Data size: 58120 Basic stats: PARTIAL Column stats: NONE
GatherStats: false
Filter Operator
isSamplingPred: false
predicate: key is not null (type: boolean)
- Statistics: Num rows: 149 Data size: 58120 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 149 Data size: 58120 Basic stats: PARTIAL Column stats: NONE
Select Operator
expressions: key (type: int), value (type: string)
outputColumnNames: _col0, _col1
- Statistics: Num rows: 149 Data size: 58120 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 149 Data size: 58120 Basic stats: PARTIAL Column stats: NONE
Map Join Operator
condition map:
Inner Join 0 to 1
@@ -208,17 +208,17 @@ STAGE PLANS:
1 _col0 (type: int)
outputColumnNames: _col0, _col1, _col3
Position of Big Table: 0
- Statistics: Num rows: 163 Data size: 63932 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 163 Data size: 63932 Basic stats: PARTIAL Column stats: NONE
Select Operator
expressions: UDFToString(_col0) (type: string), _col1 (type: string), _col3 (type: string)
outputColumnNames: _col0, _col1, _col2
- Statistics: Num rows: 163 Data size: 63932 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 163 Data size: 63932 Basic stats: PARTIAL Column stats: NONE
File Output Operator
compressed: false
GlobalTableId: 1
#### A masked pattern was here ####
NumFilesPerFileSink: 1
- Statistics: Num rows: 163 Data size: 63932 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 163 Data size: 63932 Basic stats: PARTIAL Column stats: NONE
#### A masked pattern was here ####
table:
input format: org.apache.hadoop.mapred.TextInputFormat
@@ -249,12 +249,12 @@ STAGE PLANS:
Select Operator
expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
outputColumnNames: key, value1, value2
- Statistics: Num rows: 163 Data size: 63932 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 163 Data size: 63932 Basic stats: PARTIAL Column stats: NONE
Group By Operator
aggregations: compute_stats(key, 'hll'), compute_stats(value1, 'hll'), compute_stats(value2, 'hll')
mode: hash
outputColumnNames: _col0, _col1, _col2
- Statistics: Num rows: 1 Data size: 1320 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
File Output Operator
compressed: false
GlobalTableId: 0
@@ -426,7 +426,7 @@ STAGE PLANS:
Reduce Output Operator
null sort order:
sort order:
- Statistics: Num rows: 1 Data size: 1320 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
tag: -1
value expressions: _col0 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col1 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col2 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>)
auto parallelism: false
@@ -463,13 +463,13 @@ STAGE PLANS:
aggregations: compute_stats(VALUE._col0), compute_stats(VALUE._col1), compute_stats(VALUE._col2)
mode: mergepartial
outputColumnNames: _col0, _col1, _col2
- Statistics: Num rows: 1 Data size: 1320 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
File Output Operator
compressed: false
GlobalTableId: 0
#### A masked pattern was here ####
NumFilesPerFileSink: 1
- Statistics: Num rows: 1 Data size: 1320 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
#### A masked pattern was here ####
table:
input format: org.apache.hadoop.mapred.SequenceFileInputFormat
@@ -612,16 +612,16 @@ STAGE PLANS:
TableScan
alias: b
filterExpr: ((ds = '2008-04-08') and key is not null) (type: boolean)
- Statistics: Num rows: 78 Data size: 30620 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 78 Data size: 30620 Basic stats: PARTIAL Column stats: NONE
GatherStats: false
Filter Operator
isSamplingPred: false
predicate: key is not null (type: boolean)
- Statistics: Num rows: 78 Data size: 30620 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 78 Data size: 30620 Basic stats: PARTIAL Column stats: NONE
Select Operator
expressions: key (type: int), value (type: string)
outputColumnNames: _col0, _col1
- Statistics: Num rows: 78 Data size: 30620 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 78 Data size: 30620 Basic stats: PARTIAL Column stats: NONE
HashTable Sink Operator
keys:
0 _col0 (type: int)
@@ -634,16 +634,16 @@ STAGE PLANS:
TableScan
alias: a
filterExpr: key is not null (type: boolean)
- Statistics: Num rows: 149 Data size: 58120 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 149 Data size: 58120 Basic stats: PARTIAL Column stats: NONE
GatherStats: false
Filter Operator
isSamplingPred: false
predicate: key is not null (type: boolean)
- Statistics: Num rows: 149 Data size: 58120 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 149 Data size: 58120 Basic stats: PARTIAL Column stats: NONE
Select Operator
expressions: key (type: int), value (type: string)
outputColumnNames: _col0, _col1
- Statistics: Num rows: 149 Data size: 58120 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 149 Data size: 58120 Basic stats: PARTIAL Column stats: NONE
Map Join Operator
condition map:
Inner Join 0 to 1
@@ -652,17 +652,17 @@ STAGE PLANS:
1 _col0 (type: int)
outputColumnNames: _col0, _col1, _col3
Position of Big Table: 0
- Statistics: Num rows: 163 Data size: 63932 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 163 Data size: 63932 Basic stats: PARTIAL Column stats: NONE
Select Operator
expressions: UDFToString(_col0) (type: string), _col1 (type: string), _col3 (type: string)
outputColumnNames: _col0, _col1, _col2
- Statistics: Num rows: 163 Data size: 63932 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 163 Data size: 63932 Basic stats: PARTIAL Column stats: NONE
File Output Operator
compressed: false
GlobalTableId: 1
#### A masked pattern was here ####
NumFilesPerFileSink: 1
- Statistics: Num rows: 163 Data size: 63932 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 163 Data size: 63932 Basic stats: PARTIAL Column stats: NONE
#### A masked pattern was here ####
table:
input format: org.apache.hadoop.mapred.TextInputFormat
@@ -693,12 +693,12 @@ STAGE PLANS:
Select Operator
expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
outputColumnNames: key, value1, value2
- Statistics: Num rows: 163 Data size: 63932 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 163 Data size: 63932 Basic stats: PARTIAL Column stats: NONE
Group By Operator
aggregations: compute_stats(key, 'hll'), compute_stats(value1, 'hll'), compute_stats(value2, 'hll')
mode: hash
outputColumnNames: _col0, _col1, _col2
- Statistics: Num rows: 1 Data size: 1320 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
File Output Operator
compressed: false
GlobalTableId: 0
@@ -870,7 +870,7 @@ STAGE PLANS:
Reduce Output Operator
null sort order:
sort order:
- Statistics: Num rows: 1 Data size: 1320 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
tag: -1
value expressions: _col0 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col1 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col2 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>)
auto parallelism: false
@@ -907,13 +907,13 @@ STAGE PLANS:
aggregations: compute_stats(VALUE._col0), compute_stats(VALUE._col1), compute_stats(VALUE._col2)
mode: mergepartial
outputColumnNames: _col0, _col1, _col2
- Statistics: Num rows: 1 Data size: 1320 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
File Output Operator
compressed: false
GlobalTableId: 0
#### A masked pattern was here ####
NumFilesPerFileSink: 1
- Statistics: Num rows: 1 Data size: 1320 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
#### A masked pattern was here ####
table:
input format: org.apache.hadoop.mapred.SequenceFileInputFormat