diff --git common/src/java/org/apache/hadoop/hive/common/StatsSetupConst.java common/src/java/org/apache/hadoop/hive/common/StatsSetupConst.java index cd4beeb..34865a8 100644 --- common/src/java/org/apache/hadoop/hive/common/StatsSetupConst.java +++ common/src/java/org/apache/hadoop/hive/common/StatsSetupConst.java @@ -55,6 +55,8 @@ public String getPublisher(Configuration conf) { 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"; } }, diff --git data/conf/spark/hive-site.xml data/conf/spark/hive-site.xml index df1dc9c..f3d7b40 100644 --- data/conf/spark/hive-site.xml +++ data/conf/spark/hive-site.xml @@ -185,7 +185,7 @@ hive.stats.dbclass - counter + fs The default storatge that stores temporary hive statistics. Currently, jdbc, hbase and counter type is supported diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkClient.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkClient.java index ccf2038..5766787 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkClient.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkClient.java @@ -25,14 +25,13 @@ import org.apache.hadoop.conf.Configuration; 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.ql.Context; import org.apache.hadoop.hive.ql.DriverContext; import org.apache.hadoop.hive.ql.exec.Utilities; -import org.apache.hadoop.hive.ql.exec.spark.counter.SparkCounter; -import org.apache.hadoop.hive.ql.exec.spark.counter.SparkCounterGroup; import org.apache.hadoop.hive.ql.exec.spark.counter.SparkCounters; -import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobMonitor; +import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobRef; import org.apache.hadoop.hive.ql.exec.spark.status.impl.JobStateListener; import org.apache.hadoop.hive.ql.exec.spark.status.impl.SimpleSparkJobStatus; import org.apache.hadoop.hive.ql.io.HiveKey; @@ -151,7 +150,7 @@ private SparkConf initiateSparkConf(Configuration hiveConf) { return sparkConf; } - public int execute(DriverContext driverContext, SparkWork sparkWork) { + public SparkJobRef execute(DriverContext driverContext, SparkWork sparkWork) throws Exception { Context ctx = driverContext.getCtx(); HiveConf hiveConf = (HiveConf) ctx.getConf(); refreshLocalResources(sparkWork, hiveConf); @@ -159,49 +158,35 @@ public int execute(DriverContext driverContext, SparkWork sparkWork) { // Create temporary scratch dir Path emptyScratchDir; - try { - emptyScratchDir = ctx.getMRTmpPath(); - FileSystem fs = emptyScratchDir.getFileSystem(jobConf); - fs.mkdirs(emptyScratchDir); - } catch (IOException e) { - LOG.error("Error launching map-reduce job", e); - return 5; - } + emptyScratchDir = ctx.getMRTmpPath(); + FileSystem fs = emptyScratchDir.getFileSystem(jobConf); + fs.mkdirs(emptyScratchDir); SparkCounters sparkCounters = new SparkCounters(sc, hiveConf); + List prefixes = sparkWork.getRequiredCounterPrefix(); + // register spark counters before submit spark job. + if (prefixes != null) { + for (String prefix : prefixes) { + sparkCounters.createCounter(prefix, StatsSetupConst.ROW_COUNT); + sparkCounters.createCounter(prefix, StatsSetupConst.RAW_DATA_SIZE); + } + } SparkReporter sparkReporter = new SparkReporter(sparkCounters); // Generate Spark plan SparkPlanGenerator gen = new SparkPlanGenerator(sc, ctx, jobConf, emptyScratchDir, sparkReporter); - SparkPlan plan; - try { - plan = gen.generate(sparkWork); - } catch (Exception e) { - LOG.error("Error generating Spark Plan", e); - return 2; - } + SparkPlan plan = gen.generate(sparkWork); // Execute generated plan. - try { - JavaPairRDD finalRDD = plan.generateGraph(); - // We use Spark RDD async action to submit job as it's the only way to get jobId now. - JavaFutureAction future = finalRDD.foreachAsync(HiveVoidFunction.getInstance()); - // An action may trigger multi jobs in Spark, we only monitor the latest job here - // until we found that Hive does trigger multi jobs. - List jobIds = future.jobIds(); - // jobIds size is always bigger than or equal with 1. - int jobId = jobIds.get(jobIds.size() - 1); - SimpleSparkJobStatus sparkJobStatus = new SimpleSparkJobStatus( - jobId, jobStateListener, jobProgressListener); - SparkJobMonitor monitor = new SparkJobMonitor(sparkJobStatus); - monitor.startMonitor(); - } catch (Exception e) { - LOG.error("Error executing Spark Plan", e); - return 1; - } - - return 0; + JavaPairRDD finalRDD = plan.generateGraph(); + // We use Spark RDD async action to submit job as it's the only way to get jobId now. + JavaFutureAction future = finalRDD.foreachAsync(HiveVoidFunction.getInstance()); + // As we always use foreach action to submit RDD graph, it would only trigger on job. + int jobId = future.jobIds().get(0); + SimpleSparkJobStatus sparkJobStatus = + new SimpleSparkJobStatus(jobId, jobStateListener, jobProgressListener, sparkCounters); + return new SparkJobRef(jobId, sparkJobStatus); } /** diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkMapRecordHandler.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkMapRecordHandler.java index 0bb554a..163a6fd 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkMapRecordHandler.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkMapRecordHandler.java @@ -105,6 +105,7 @@ public void init(JobConf job, OutputCollector output, Reporter reporter) { execContext.setLocalWork(localWork); MapredContext.init(true, new JobConf(jc)); + MapredContext.get().setReporter(reporter); mo.setExecContext(execContext); mo.initializeLocalWork(jc); diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkRecordHandler.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkRecordHandler.java index ec51864..a5d73a7 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkRecordHandler.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkRecordHandler.java @@ -48,6 +48,7 @@ public void init(JobConf job, OutputCollector output, Reporter reporter) { jc = job; MapredContext.init(false, new JobConf(jc)); + MapredContext.get().setReporter(reporter); oc = output; rp = reporter; diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java index e954153..b38ef7d 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java @@ -30,9 +30,12 @@ import org.apache.hadoop.hive.ql.exec.Operator; import org.apache.hadoop.hive.ql.exec.Task; import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.exec.spark.counter.SparkCounters; import org.apache.hadoop.hive.ql.exec.spark.session.SparkSession; import org.apache.hadoop.hive.ql.exec.spark.session.SparkSessionManager; import org.apache.hadoop.hive.ql.exec.spark.session.SparkSessionManagerImpl; +import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobMonitor; +import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobRef; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.plan.BaseWork; import org.apache.hadoop.hive.ql.plan.MapWork; @@ -47,6 +50,7 @@ private static final long serialVersionUID = 1L; private transient JobConf job; private transient ContentSummary inputSummary; + private SparkCounters sparkCounters; @Override public void initialize(HiveConf conf, QueryPlan queryPlan, DriverContext driverContext) { @@ -64,8 +68,8 @@ public int execute(DriverContext driverContext) { printConfigInfo(); sparkSessionManager = SparkSessionManagerImpl.getInstance(); sparkSession = SessionState.get().getSparkSession(); - - // Spark configurations are updated close the existing session + + // Spark configurations are updated close the existing session if(conf.getSparkConfigUpdated()){ sparkSessionManager.closeSession(sparkSession); sparkSession = null; @@ -73,12 +77,21 @@ public int execute(DriverContext driverContext) { } sparkSession = sparkSessionManager.getSession(sparkSession, conf, true); SessionState.get().setSparkSession(sparkSession); - rc = sparkSession.submit(driverContext, getWork()); + SparkWork sparkWork = getWork(); + String statsImpl = HiveConf.getVar(conf, HiveConf.ConfVars.HIVESTATSDBCLASS); + if (statsImpl.equalsIgnoreCase("counter")) { + sparkWork.setRequiredCounterPrefix(SparkUtilities.getRequiredCounterPrefix(this, db)); + } + SparkJobRef jobRef = sparkSession.submit(driverContext, sparkWork); + sparkCounters = jobRef.getSparkJobStatus().getCounter(); + SparkJobMonitor monitor = new SparkJobMonitor(jobRef.getSparkJobStatus()); + monitor.startMonitor(); + console.printInfo(sparkCounters.toString()); + rc = 0; } catch (Exception e) { LOG.error("Failed to execute spark task.", e); return 1; - } - finally { + } finally { if (sparkSession != null && sparkSessionManager != null) { rc = close(rc); try { @@ -154,6 +167,10 @@ public String getName() { return result; } + public SparkCounters getSparkCounters() { + return sparkCounters; + } + /** * Set the number of reducers for the spark work. */ diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java index 37761c9..cc8afa7 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java @@ -17,7 +17,35 @@ */ package org.apache.hadoop.hive.ql.exec.spark; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.hadoop.hive.metastore.Warehouse; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.ql.exec.FileSinkOperator; +import org.apache.hadoop.hive.ql.exec.MoveTask; +import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.hive.ql.exec.StatsTask; +import org.apache.hadoop.hive.ql.exec.TableScanOperator; +import org.apache.hadoop.hive.ql.exec.Task; +import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.io.HiveKey; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.metadata.Partition; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer; +import org.apache.hadoop.hive.ql.plan.BaseWork; +import org.apache.hadoop.hive.ql.plan.DynamicPartitionCtx; +import org.apache.hadoop.hive.ql.plan.FileSinkDesc; +import org.apache.hadoop.hive.ql.plan.LoadTableDesc; +import org.apache.hadoop.hive.ql.plan.MoveWork; +import org.apache.hadoop.hive.ql.plan.StatsWork; +import org.apache.hadoop.hive.ql.plan.TableScanDesc; import org.apache.hadoop.io.BytesWritable; /** @@ -42,4 +70,108 @@ public static BytesWritable copyBytesWritable(BytesWritable bw) { return copy; } + public static List getRequiredCounterPrefix(SparkTask sparkTask, Hive db) + throws HiveException, MetaException { + + List prefixs = new LinkedList(); + List works = sparkTask.getWork().getAllWork(); + for (BaseWork baseWork : works) { + Set> operators = baseWork.getAllOperators(); + for (Operator operator : operators) { + if (operator instanceof TableScanOperator) { + TableScanOperator tableScanOperator = (TableScanOperator) operator; + TableScanDesc tableScanDesc = tableScanOperator.getConf(); + + if (tableScanDesc.isGatherStats()) { + List> childTasks = getChildTasks(sparkTask); + for (Task task : childTasks) { + if (task instanceof StatsTask) { + StatsTask statsTask = (StatsTask) task; + StatsWork statsWork = statsTask.getWork(); + // ANALYZE command + BaseSemanticAnalyzer.tableSpec tblSpec = statsWork.getTableSpecs(); + Table table = tblSpec.tableHandle; + if (!table.isPartitioned()) { + prefixs.add(tableScanDesc.getStatsAggPrefix()); // non-partitioned + } else { + for (Partition partition : tblSpec.partitions) { + String aggrPrefix = getAggregationPrefix( + table, partition.getSpec(), tableScanDesc.getMaxStatsKeyPrefixLength()); + prefixs.add(aggrPrefix); + } + } + } + } + } + } else if (operator instanceof FileSinkOperator) { + FileSinkOperator fileSinkOperator = (FileSinkOperator) operator; + FileSinkDesc fileSinkDesc = fileSinkOperator.getConf(); + + if (fileSinkDesc.isGatherStats()) { + List> childTasks = getChildTasks(sparkTask); + for (Task task : childTasks) { + if (task instanceof MoveTask) { + MoveTask moveTask = (MoveTask) task; + MoveWork moveWork = moveTask.getWork(); + + // INSERT OVERWRITE command + LoadTableDesc tbd = moveWork.getLoadTableWork(); + Table table = db.getTable(tbd.getTable().getTableName()); + if (!table.isPartitioned()) { + prefixs.add( + getAggregationPrefix(table, null, fileSinkDesc.getMaxStatsKeyPrefixLength())); + } else { + DynamicPartitionCtx dpCtx = tbd.getDPCtx(); + if (dpCtx == null || dpCtx.getNumDPCols() == 0) { + // static partition + Map partitionSpec = tbd.getPartitionSpec(); + if (partitionSpec != null && !partitionSpec.isEmpty()) { + String aggrPrefix = getAggregationPrefix( + table, partitionSpec, fileSinkDesc.getMaxStatsKeyPrefixLength()); + prefixs.add(aggrPrefix); + } + } else { + // dynamic partition + } + } + } + } + } + } + } + } + return prefixs; + } + + private static String getAggregationPrefix(Table table, Map partitionSpec, int maxKeyLength) + throws MetaException { + StringBuilder prefix = new StringBuilder(); + // prefix is of the form dbName.tblName + prefix.append(table.getDbName()).append('.').append(table.getTableName()); + if (partitionSpec != null) { + return Utilities.join(prefix.toString(), Warehouse.makePartPath(partitionSpec)); + } + return Utilities.getHashedStatsPrefix(prefix.toString(), maxKeyLength); + } + + private static List> getChildTasks( + Task rootTask) { + + List> tasks = new ArrayList>(); + fillChildTasks(tasks, rootTask); + return tasks; + } + + private static void fillChildTasks( + List> tasks, + Task rootTask) { + + List> childTasks = rootTask.getChildTasks(); + tasks.add(rootTask); + if (childTasks != null) { + for (Task task : childTasks) { + fillChildTasks(tasks, task); + } + } + } } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/counter/SparkCounters.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/counter/SparkCounters.java index 447d964..45ccada 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/counter/SparkCounters.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/counter/SparkCounters.java @@ -21,6 +21,8 @@ import java.util.HashMap; import java.util.Map; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.exec.FilterOperator; @@ -44,6 +46,7 @@ * 3. Hive could only get Counter value at driver side. */ public class SparkCounters implements Serializable { + private static final Log LOG = LogFactory.getLog(SparkCounters.class); private Map sparkCounterGroups; @@ -90,20 +93,22 @@ public void increment(Enum key, long incrValue) { public void increment(String groupName, String counterName, long value) { SparkCounter counter = getGroup(groupName).getCounter(counterName); if (counter == null) { - throw new RuntimeException( + LOG.error( String.format("counter[%s, %s] has not initialized before.", groupName, counterName)); + } else { + counter.increment(value); } - counter.increment(value); } public long getValue(String groupName, String counterName) { SparkCounter counter = getGroup(groupName).getCounter(counterName); if (counter == null) { - throw new RuntimeException( + LOG.error( String.format("counter[%s, %s] has not initialized before.", groupName, counterName)); + return 0; + } else { + return counter.getValue(); } - - return counter.getValue(); } public SparkCounter getCounter(String groupName, String counterName) { @@ -127,4 +132,28 @@ private SparkCounterGroup getGroup(String groupName) { public Map getSparkCounterGroups() { return sparkCounterGroups; } + + public String toString() { + StringBuilder sb = new StringBuilder(); + Map groups = getSparkCounterGroups(); + if (groups != null) { + for(Map.Entry groupEntry : groups.entrySet()) { + String groupName = groupEntry.getKey(); + SparkCounterGroup group = groupEntry.getValue(); + sb.append(groupName).append("\n"); + Map counters = group.getSparkCounters(); + for (Map.Entry counterEntry : counters.entrySet()) { + String counterName = counterEntry.getKey(); + SparkCounter counter = counterEntry.getValue(); + sb.append("\t") + .append(counterName) + .append(": ") + .append(counter.getValue()) + .append("\n"); + } + } + } + + return sb.toString(); + } } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSession.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSession.java index 02456fc..044cf7c 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSession.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSession.java @@ -19,6 +19,7 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobRef; import org.apache.hadoop.hive.ql.plan.SparkWork; public interface SparkSession { @@ -32,7 +33,7 @@ * @param driverContext * @param sparkWork */ - public int submit(DriverContext driverContext, SparkWork sparkWork); + public SparkJobRef submit(DriverContext driverContext, SparkWork sparkWork) throws Exception; /** * Is the session open and ready to submit jobs? diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionImpl.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionImpl.java index eefd4be..51e0510 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionImpl.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionImpl.java @@ -21,6 +21,7 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.DriverContext; import org.apache.hadoop.hive.ql.exec.spark.SparkClient; +import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobRef; import org.apache.hadoop.hive.ql.plan.SparkWork; import java.util.UUID; @@ -46,7 +47,7 @@ public void open(HiveConf conf) { } @Override - public int submit(DriverContext driverContext, SparkWork sparkWork) { + public SparkJobRef submit(DriverContext driverContext, SparkWork sparkWork) throws Exception { Preconditions.checkState(isOpen, "Session is not open. Can't submit jobs."); sparkClient = SparkClient.getInstance(driverContext.getCtx().getConf()); return sparkClient.execute(driverContext, sparkWork); diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobRef.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobRef.java new file mode 100644 index 0000000..bf43b6e --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobRef.java @@ -0,0 +1,52 @@ +/** + * 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.exec.spark.status; + +public class SparkJobRef { + + private int jobId; + + private SparkJobStatus sparkJobStatus; + + public SparkJobRef() {} + + public SparkJobRef(int jobId) { + this.jobId = jobId; + } + + public SparkJobRef(int jobId, SparkJobStatus sparkJobStatus) { + this.jobId = jobId; + this.sparkJobStatus = sparkJobStatus; + } + + public int getJobId() { + return jobId; + } + + public void setJobId(int jobId) { + this.jobId = jobId; + } + + public SparkJobStatus getSparkJobStatus() { + return sparkJobStatus; + } + + public void setSparkJobStatus(SparkJobStatus sparkJobStatus) { + this.sparkJobStatus = sparkJobStatus; + } +} diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobStatus.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobStatus.java index 25cb60f..a450af4 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobStatus.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobStatus.java @@ -17,6 +17,8 @@ */ package org.apache.hadoop.hive.ql.exec.spark.status; +import org.apache.hadoop.hive.ql.exec.spark.counter.SparkCounters; + import java.util.Map; /** @@ -32,4 +34,6 @@ public Map getSparkStageProgress(); + public SparkCounters getCounter(); + } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/SimpleSparkJobStatus.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/SimpleSparkJobStatus.java index 6570b0c..db2eca4 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/SimpleSparkJobStatus.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/SimpleSparkJobStatus.java @@ -22,6 +22,7 @@ import java.util.List; import java.util.Map; +import org.apache.hadoop.hive.ql.exec.spark.counter.SparkCounters; import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobState; import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobStatus; import org.apache.hadoop.hive.ql.exec.spark.status.SparkStageProgress; @@ -40,15 +41,18 @@ private int jobId; private JobStateListener jobStateListener; private JobProgressListener jobProgressListener; + private SparkCounters sparkCounters; public SimpleSparkJobStatus( int jobId, JobStateListener stateListener, - JobProgressListener progressListener) { + JobProgressListener progressListener, + SparkCounters sparkCounters) { this.jobId = jobId; this.jobStateListener = stateListener; this.jobProgressListener = progressListener; + this.sparkCounters = sparkCounters; } @Override @@ -111,6 +115,11 @@ public SparkJobState getState() { return stageProgresses; } + @Override + public SparkCounters getCounter() { + return sparkCounters; + } + private List getStageInfo(int stageId) { List stageInfos = new LinkedList(); diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/SparkWork.java ql/src/java/org/apache/hadoop/hive/ql/plan/SparkWork.java index 26428e8..66fd6b6 100644 --- ql/src/java/org/apache/hadoop/hive/ql/plan/SparkWork.java +++ ql/src/java/org/apache/hadoop/hive/ql/plan/SparkWork.java @@ -54,10 +54,13 @@ protected final Map, SparkEdgeProperty> edgeProperties = new HashMap, SparkEdgeProperty>(); + private List requiredCounterPrefix; + public SparkWork(String name) { this.name = name + ":" + (++counter); } + @Explain(displayName = "DagName") public String getName() { return name; @@ -173,6 +176,14 @@ public void disconnect(BaseWork a, BaseWork b) { return new HashSet(leaves); } + public void setRequiredCounterPrefix(List requiredCounterPrefix) { + this.requiredCounterPrefix = requiredCounterPrefix; + } + + public List getRequiredCounterPrefix() { + return requiredCounterPrefix; + } + /** * getParents returns all the nodes with edges leading into work */ @@ -300,5 +311,5 @@ public int compareTo(Dependency o) { } return result; } - + } diff --git ql/src/java/org/apache/hadoop/hive/ql/stats/CounterStatsAggregatorSpark.java ql/src/java/org/apache/hadoop/hive/ql/stats/CounterStatsAggregatorSpark.java new file mode 100644 index 0000000..0847ce7 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/stats/CounterStatsAggregatorSpark.java @@ -0,0 +1,62 @@ +/** + * 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.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +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.hadoop.hive.ql.exec.spark.counter.SparkCounters; + +public class CounterStatsAggregatorSpark + implements StatsAggregator, StatsCollectionTaskIndependent { + + private static final Log LOG = LogFactory.getLog(CounterStatsAggregatorSpark.class); + + private SparkCounters sparkCounters; + + @Override + public boolean connect(Configuration hconf, Task sourceTask) { + SparkTask task = (SparkTask)sourceTask; + 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() { + return true; + } + + @Override + public boolean cleanUp(String keyPrefix) { + return true; + } +} diff --git ql/src/java/org/apache/hadoop/hive/ql/stats/CounterStatsPublisher.java ql/src/java/org/apache/hadoop/hive/ql/stats/CounterStatsPublisher.java index 0b648aa..6f4cd53 100644 --- ql/src/java/org/apache/hadoop/hive/ql/stats/CounterStatsPublisher.java +++ ql/src/java/org/apache/hadoop/hive/ql/stats/CounterStatsPublisher.java @@ -52,8 +52,9 @@ public boolean publishStat(String fileID, Map stats) { for (Map.Entry entry : stats.entrySet()) { try { reporter.incrCounter(fileID, entry.getKey(), Long.valueOf(entry.getValue())); - } catch (NumberFormatException e) { - LOG.error("Invalid counter value " + entry.getValue() + " for " + entry.getKey()); + } catch (Exception e) { + LOG.error("Failed to increment counter value " + entry.getValue() + " for " + entry.getKey()); + return false; } } return true; diff --git ql/src/test/results/clientpositive/spark/add_part_multiple.q.out ql/src/test/results/clientpositive/spark/add_part_multiple.q.out index 264a08d..975baa5 100644 --- ql/src/test/results/clientpositive/spark/add_part_multiple.q.out +++ ql/src/test/results/clientpositive/spark/add_part_multiple.q.out @@ -68,10 +68,6 @@ PREHOOK: Output: default@add_part_test@ds=2010-01-01 PREHOOK: Output: default@add_part_test@ds=2010-02-01 PREHOOK: Output: default@add_part_test@ds=2010-03-01 PREHOOK: Output: default@add_part_test@ds=2010-04-01 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: from src TABLESAMPLE (1 ROWS) insert into table add_part_test PARTITION (ds='2010-01-01') select 100,100 insert into table add_part_test PARTITION (ds='2010-02-01') select 200,200 diff --git ql/src/test/results/clientpositive/spark/alter_merge_orc.q.out ql/src/test/results/clientpositive/spark/alter_merge_orc.q.out index 0fe395c..78f40f4 100644 --- ql/src/test/results/clientpositive/spark/alter_merge_orc.q.out +++ ql/src/test/results/clientpositive/spark/alter_merge_orc.q.out @@ -10,7 +10,6 @@ PREHOOK: query: insert overwrite table src_orc_merge_test select * from src PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@src_orc_merge_test -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table src_orc_merge_test select * from src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -21,7 +20,6 @@ PREHOOK: query: insert into table src_orc_merge_test select * from src PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@src_orc_merge_test -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert into table src_orc_merge_test select * from src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -32,7 +30,6 @@ PREHOOK: query: insert into table src_orc_merge_test select * from src PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@src_orc_merge_test -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert into table src_orc_merge_test select * from src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -136,7 +133,6 @@ PREHOOK: query: insert overwrite table src_orc_merge_test_part partition (ds='20 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@src_orc_merge_test_part@ds=2011 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table src_orc_merge_test_part partition (ds='2011') select * from src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -147,7 +143,6 @@ PREHOOK: query: insert into table src_orc_merge_test_part partition (ds='2011') PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@src_orc_merge_test_part@ds=2011 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert into table src_orc_merge_test_part partition (ds='2011') select * from src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -158,7 +153,6 @@ PREHOOK: query: insert into table src_orc_merge_test_part partition (ds='2011') PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@src_orc_merge_test_part@ds=2011 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert into table src_orc_merge_test_part partition (ds='2011') select * from src POSTHOOK: type: QUERY POSTHOOK: Input: default@src diff --git ql/src/test/results/clientpositive/spark/alter_merge_stats_orc.q.out ql/src/test/results/clientpositive/spark/alter_merge_stats_orc.q.out index 3e64d36..f8486ad 100644 --- ql/src/test/results/clientpositive/spark/alter_merge_stats_orc.q.out +++ ql/src/test/results/clientpositive/spark/alter_merge_stats_orc.q.out @@ -10,7 +10,6 @@ PREHOOK: query: insert overwrite table src_orc_merge_test_stat select * from src PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@src_orc_merge_test_stat -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table src_orc_merge_test_stat select * from src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -21,7 +20,6 @@ PREHOOK: query: insert into table src_orc_merge_test_stat select * from src PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@src_orc_merge_test_stat -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert into table src_orc_merge_test_stat select * from src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -32,7 +30,6 @@ PREHOOK: query: insert into table src_orc_merge_test_stat select * from src PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@src_orc_merge_test_stat -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert into table src_orc_merge_test_stat select * from src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -181,7 +178,6 @@ PREHOOK: query: insert overwrite table src_orc_merge_test_part_stat partition (d PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@src_orc_merge_test_part_stat@ds=2011 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table src_orc_merge_test_part_stat partition (ds='2011') select * from src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -192,7 +188,6 @@ PREHOOK: query: insert into table src_orc_merge_test_part_stat partition (ds='20 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@src_orc_merge_test_part_stat@ds=2011 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert into table src_orc_merge_test_part_stat partition (ds='2011') select * from src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -203,7 +198,6 @@ PREHOOK: query: insert into table src_orc_merge_test_part_stat partition (ds='20 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@src_orc_merge_test_part_stat@ds=2011 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert into table src_orc_merge_test_part_stat partition (ds='2011') select * from src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -253,6 +247,8 @@ Protect Mode: None Partition Parameters: COLUMN_STATS_ACCURATE true numFiles 3 + numRows 1500 + rawDataSize 141000 totalSize 7488 #### A masked pattern was here #### diff --git ql/src/test/results/clientpositive/spark/auto_smb_mapjoin_14.q.out ql/src/test/results/clientpositive/spark/auto_smb_mapjoin_14.q.out index e64d4fb..193cbae 100644 --- ql/src/test/results/clientpositive/spark/auto_smb_mapjoin_14.q.out +++ ql/src/test/results/clientpositive/spark/auto_smb_mapjoin_14.q.out @@ -23,7 +23,6 @@ select * from src where key < 10 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@tbl1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table tbl1 select * from src where key < 10 POSTHOOK: type: QUERY @@ -36,7 +35,6 @@ select * from src where key < 10 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@tbl2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table tbl2 select * from src where key < 10 POSTHOOK: type: QUERY @@ -72,28 +70,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: a - Statistics: Num rows: 20 Data size: 80 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE Map 4 Map Operator Tree: TableScan alias: b - Statistics: Num rows: 20 Data size: 80 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator @@ -102,9 +100,9 @@ STAGE PLANS: condition expressions: 0 1 - Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash @@ -193,28 +191,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 20 Data size: 80 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE Map 5 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 20 Data size: 80 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator @@ -224,22 +222,22 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} 1 outputColumnNames: _col0 - Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int) outputColumnNames: _col0 - Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() keys: _col0 (type: int) mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Reducer 3 Reduce Operator Tree: @@ -248,9 +246,9 @@ STAGE PLANS: keys: KEY._col0 (type: int) mode: mergepartial outputColumnNames: _col0, _col1 - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 15 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 15 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash @@ -367,54 +365,54 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 20 Data size: 80 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE Map 5 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 20 Data size: 80 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE Map 6 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 20 Data size: 80 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE Map 9 Map Operator Tree: TableScan alias: b - Statistics: Num rows: 20 Data size: 80 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator @@ -424,22 +422,22 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} 1 outputColumnNames: _col0 - Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int) outputColumnNames: _col0 - Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() keys: _col0 (type: int) mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Reducer 3 Reduce Operator Tree: @@ -448,16 +446,16 @@ STAGE PLANS: keys: KEY._col0 (type: int) mode: mergepartial outputColumnNames: _col0, _col1 - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 15 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: bigint) outputColumnNames: _col0, _col1 - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 15 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 15 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Reducer 4 Reduce Operator Tree: @@ -468,14 +466,14 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 {VALUE._col0} outputColumnNames: _col0, _col1, _col3 - Statistics: Num rows: 5 Data size: 22 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: bigint), _col3 (type: bigint) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 5 Data size: 22 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 5 Data size: 22 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 16 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -489,22 +487,22 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} 1 outputColumnNames: _col0 - Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int) outputColumnNames: _col0 - Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() keys: _col0 (type: int) mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Reducer 8 Reduce Operator Tree: @@ -513,16 +511,16 @@ STAGE PLANS: keys: KEY._col0 (type: int) mode: mergepartial outputColumnNames: _col0, _col1 - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 15 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: bigint) outputColumnNames: _col0, _col1 - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 15 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 15 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Stage: Stage-0 @@ -609,36 +607,36 @@ STAGE PLANS: Map Operator Tree: TableScan alias: a - Statistics: Num rows: 20 Data size: 80 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: ((key < 6) and key is not null) (type: boolean) - Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int) outputColumnNames: _col0 - Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE Map 4 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 20 Data size: 80 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: ((key < 6) and key is not null) (type: boolean) - Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int) outputColumnNames: _col0 - Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator @@ -647,9 +645,9 @@ STAGE PLANS: condition expressions: 0 1 - Statistics: Num rows: 3 Data size: 13 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 15 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 3 Data size: 13 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 15 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash @@ -747,32 +745,32 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 20 Data size: 80 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE Map 4 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 20 Data size: 80 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (((key < 8) and (key < 6)) and key is not null) (type: boolean) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int) outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator @@ -781,9 +779,9 @@ STAGE PLANS: condition expressions: 0 1 - Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash @@ -905,36 +903,36 @@ STAGE PLANS: Map Operator Tree: TableScan alias: a - Statistics: Num rows: 20 Data size: 80 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (((key < 8) and (key < 6)) and key is not null) (type: boolean) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int) outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Map 4 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 20 Data size: 80 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (((key < 8) and (key < 6)) and key is not null) (type: boolean) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int) outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator @@ -943,9 +941,9 @@ STAGE PLANS: condition expressions: 0 1 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash @@ -1057,36 +1055,36 @@ STAGE PLANS: Map Operator Tree: TableScan alias: a - Statistics: Num rows: 20 Data size: 80 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: ((key < 8) and key is not null) (type: boolean) - Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int) outputColumnNames: _col0 - Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE Map 4 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 20 Data size: 80 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: ((key < 8) and key is not null) (type: boolean) - Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int) outputColumnNames: _col0 - Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator @@ -1095,9 +1093,9 @@ STAGE PLANS: condition expressions: 0 1 - Statistics: Num rows: 3 Data size: 13 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 15 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 3 Data size: 13 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 15 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash @@ -1185,36 +1183,36 @@ STAGE PLANS: Map Operator Tree: TableScan alias: a - Statistics: Num rows: 20 Data size: 80 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: (key + 1) (type: int) outputColumnNames: _col0 - Statistics: Num rows: 20 Data size: 80 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: _col0 is not null (type: boolean) - Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE Map 4 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 20 Data size: 80 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: (key + 1) (type: int) outputColumnNames: _col0 - Statistics: Num rows: 20 Data size: 80 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: _col0 is not null (type: boolean) - Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator @@ -1223,9 +1221,9 @@ STAGE PLANS: condition expressions: 0 1 - Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash @@ -1309,32 +1307,32 @@ STAGE PLANS: Map Operator Tree: TableScan alias: a - Statistics: Num rows: 20 Data size: 80 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: ((key < 6) and key is not null) (type: boolean) - Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int) outputColumnNames: _col0 - Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE Map 4 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 20 Data size: 80 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator @@ -1343,9 +1341,9 @@ STAGE PLANS: condition expressions: 0 1 - Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash @@ -1435,53 +1433,53 @@ STAGE PLANS: Map Operator Tree: TableScan alias: a - Statistics: Num rows: 20 Data size: 80 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: ((key < 6) and key is not null) (type: boolean) - Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int) outputColumnNames: _col0 - Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE Map 4 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 20 Data size: 80 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: ((key < 6) and key is not null) (type: boolean) - Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int) outputColumnNames: _col0 - Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE Map 5 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 20 Data size: 80 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: ((key < 6) and key is not null) (type: boolean) - Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int) outputColumnNames: _col0 - Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator @@ -1492,9 +1490,9 @@ STAGE PLANS: 0 1 2 - Statistics: Num rows: 6 Data size: 26 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 4 Data size: 30 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 6 Data size: 26 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 4 Data size: 30 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash @@ -1600,32 +1598,32 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 20 Data size: 80 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE Map 4 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 20 Data size: 80 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (((key < 8) and (key < 6)) and key is not null) (type: boolean) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int) outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator @@ -1634,9 +1632,9 @@ STAGE PLANS: condition expressions: 0 1 - Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash @@ -1755,29 +1753,29 @@ STAGE PLANS: Map Operator Tree: TableScan alias: a - Statistics: Num rows: 0 Data size: 80 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Map 3 Map Operator Tree: TableScan alias: b - Statistics: Num rows: 0 Data size: 80 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Reducer 2 Reduce Operator Tree: @@ -1788,18 +1786,18 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 {VALUE._col0} outputColumnNames: _col0, _col1, _col6 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col6 (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -1808,10 +1806,10 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -1857,8 +1855,6 @@ PREHOOK: Input: default@tbl1 PREHOOK: Input: default@tbl2 PREHOOK: Output: default@dest1 PREHOOK: Output: default@dest2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: from ( select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key ) subq1 @@ -1988,29 +1984,29 @@ STAGE PLANS: Map Operator Tree: TableScan alias: a - Statistics: Num rows: 0 Data size: 80 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Map 4 Map Operator Tree: TableScan alias: b - Statistics: Num rows: 20 Data size: 80 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator @@ -2020,18 +2016,18 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 outputColumnNames: _col0, _col1 - Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -2040,18 +2036,18 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int) outputColumnNames: _col0 - Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() keys: _col0 (type: int) mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Reducer 3 Reduce Operator Tree: @@ -2060,14 +2056,14 @@ STAGE PLANS: keys: KEY._col0 (type: int) mode: mergepartial outputColumnNames: _col0, _col1 - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 15 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), UDFToInteger(_col1) (type: int) outputColumnNames: _col0, _col1 - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 15 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 15 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -2113,8 +2109,6 @@ PREHOOK: Input: default@tbl1 PREHOOK: Input: default@tbl2 PREHOOK: Output: default@dest1 PREHOOK: Output: default@dest2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: from ( select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key ) subq1 diff --git ql/src/test/results/clientpositive/spark/avro_decimal_native.q.out ql/src/test/results/clientpositive/spark/avro_decimal_native.q.out index db5b0f2..60b4ccc 100644 --- ql/src/test/results/clientpositive/spark/avro_decimal_native.q.out +++ ql/src/test/results/clientpositive/spark/avro_decimal_native.q.out @@ -71,7 +71,6 @@ PREHOOK: query: INSERT OVERWRITE TABLE avro_dec SELECT name, value FROM dec PREHOOK: type: QUERY PREHOOK: Input: default@dec PREHOOK: Output: default@avro_dec -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE avro_dec SELECT name, value FROM dec POSTHOOK: type: QUERY POSTHOOK: Input: default@dec diff --git ql/src/test/results/clientpositive/spark/bucket2.q.out ql/src/test/results/clientpositive/spark/bucket2.q.out index b1b2997..2fec831 100644 --- ql/src/test/results/clientpositive/spark/bucket2.q.out +++ ql/src/test/results/clientpositive/spark/bucket2.q.out @@ -186,7 +186,6 @@ select * from src PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@bucket2_1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table bucket2_1 select * from src POSTHOOK: type: QUERY @@ -210,14 +209,14 @@ STAGE PLANS: Processor Tree: TableScan alias: s - Statistics: Num rows: 55 Data size: 5812 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (((hash(key) & 2147483647) % 2) = 0) (type: boolean) - Statistics: Num rows: 27 Data size: 2853 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int), value (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 27 Data size: 2853 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE ListSink PREHOOK: query: select * from bucket2_1 tablesample (bucket 1 out of 2) s diff --git ql/src/test/results/clientpositive/spark/bucket3.q.out ql/src/test/results/clientpositive/spark/bucket3.q.out index 019c11a..af340d4 100644 --- ql/src/test/results/clientpositive/spark/bucket3.q.out +++ ql/src/test/results/clientpositive/spark/bucket3.q.out @@ -197,7 +197,6 @@ select * from src PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@bucket3_1@ds=1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table bucket3_1 partition (ds='1') select * from src POSTHOOK: type: QUERY @@ -210,7 +209,6 @@ select * from src PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@bucket3_1@ds=2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table bucket3_1 partition (ds='2') select * from src POSTHOOK: type: QUERY @@ -234,14 +232,14 @@ STAGE PLANS: Processor Tree: TableScan alias: s - Statistics: Num rows: 55 Data size: 5812 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (((hash(key) & 2147483647) % 2) = 0) (type: boolean) - Statistics: Num rows: 27 Data size: 2853 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int), value (type: string), ds (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 27 Data size: 2853 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE ListSink PREHOOK: query: select * from bucket3_1 tablesample (bucket 1 out of 2) s where ds = '1' diff --git ql/src/test/results/clientpositive/spark/bucket4.q.out ql/src/test/results/clientpositive/spark/bucket4.q.out index 2cbab11..2eda107 100644 --- ql/src/test/results/clientpositive/spark/bucket4.q.out +++ ql/src/test/results/clientpositive/spark/bucket4.q.out @@ -185,7 +185,6 @@ select * from src PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@bucket4_1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table bucket4_1 select * from src POSTHOOK: type: QUERY @@ -209,14 +208,14 @@ STAGE PLANS: Processor Tree: TableScan alias: s - Statistics: Num rows: 55 Data size: 5812 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (((hash(key) & 2147483647) % 2) = 0) (type: boolean) - Statistics: Num rows: 27 Data size: 2853 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int), value (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 27 Data size: 2853 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE ListSink PREHOOK: query: select * from bucket4_1 tablesample (bucket 1 out of 2) s diff --git ql/src/test/results/clientpositive/spark/column_access_stats.q.out ql/src/test/results/clientpositive/spark/column_access_stats.q.out index 7ffe3e3..42f6629 100644 --- ql/src/test/results/clientpositive/spark/column_access_stats.q.out +++ ql/src/test/results/clientpositive/spark/column_access_stats.q.out @@ -343,8 +343,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@t1 PREHOOK: Output: default@t2 PREHOOK: Output: default@t3 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to Table:default@t1 Columns:key,val diff --git ql/src/test/results/clientpositive/spark/create_merge_compressed.q.out ql/src/test/results/clientpositive/spark/create_merge_compressed.q.out index 7c27c6a..58f517c 100644 --- ql/src/test/results/clientpositive/spark/create_merge_compressed.q.out +++ ql/src/test/results/clientpositive/spark/create_merge_compressed.q.out @@ -26,7 +26,6 @@ PREHOOK: query: insert into table tgt_rc_merge_test select * from src_rc_merge_t PREHOOK: type: QUERY PREHOOK: Input: default@src_rc_merge_test PREHOOK: Output: default@tgt_rc_merge_test -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert into table tgt_rc_merge_test select * from src_rc_merge_test POSTHOOK: type: QUERY POSTHOOK: Input: default@src_rc_merge_test @@ -37,7 +36,6 @@ PREHOOK: query: insert into table tgt_rc_merge_test select * from src_rc_merge_t PREHOOK: type: QUERY PREHOOK: Input: default@src_rc_merge_test PREHOOK: Output: default@tgt_rc_merge_test -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert into table tgt_rc_merge_test select * from src_rc_merge_test POSTHOOK: type: QUERY POSTHOOK: Input: default@src_rc_merge_test diff --git ql/src/test/results/clientpositive/spark/custom_input_output_format.q.out ql/src/test/results/clientpositive/spark/custom_input_output_format.q.out index d7b763a..662ed1a 100644 --- ql/src/test/results/clientpositive/spark/custom_input_output_format.q.out +++ ql/src/test/results/clientpositive/spark/custom_input_output_format.q.out @@ -61,7 +61,6 @@ PREHOOK: query: INSERT OVERWRITE TABLE src1_rot13_iof SELECT * FROM src1 PREHOOK: type: QUERY PREHOOK: Input: default@src1 PREHOOK: Output: default@src1_rot13_iof -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE src1_rot13_iof SELECT * FROM src1 POSTHOOK: type: QUERY POSTHOOK: Input: default@src1 diff --git ql/src/test/results/clientpositive/spark/date_udf.q.out ql/src/test/results/clientpositive/spark/date_udf.q.out index 4018826..9b37da6 100644 --- ql/src/test/results/clientpositive/spark/date_udf.q.out +++ ql/src/test/results/clientpositive/spark/date_udf.q.out @@ -35,8 +35,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@date_udf PREHOOK: Output: default@date_udf_string -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: from src insert overwrite table date_udf select '2011-05-06' limit 1 diff --git ql/src/test/results/clientpositive/spark/disable_merge_for_bucketing.q.out ql/src/test/results/clientpositive/spark/disable_merge_for_bucketing.q.out index 590b265..0c7160d 100644 --- ql/src/test/results/clientpositive/spark/disable_merge_for_bucketing.q.out +++ ql/src/test/results/clientpositive/spark/disable_merge_for_bucketing.q.out @@ -182,7 +182,6 @@ select * from src PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@bucket2_1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table bucket2_1 select * from src POSTHOOK: type: QUERY @@ -211,28 +210,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: s - Statistics: Num rows: 55 Data size: 5812 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (((hash(key) & 2147483647) % 2) = 0) (type: boolean) - Statistics: Num rows: 27 Data size: 2853 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int), value (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 27 Data size: 2853 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) sort order: + - Statistics: Num rows: 27 Data size: 2853 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string) Reducer 2 Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 27 Data size: 2853 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 27 Data size: 2853 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat diff --git ql/src/test/results/clientpositive/spark/enforce_order.q.out ql/src/test/results/clientpositive/spark/enforce_order.q.out index 95fd12a..3e43088 100644 --- ql/src/test/results/clientpositive/spark/enforce_order.q.out +++ ql/src/test/results/clientpositive/spark/enforce_order.q.out @@ -26,7 +26,6 @@ PREHOOK: query: insert overwrite table table_asc select key, value from src PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@table_asc -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table table_asc select key, value from src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -37,7 +36,6 @@ PREHOOK: query: insert overwrite table table_desc select key, value from src PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@table_desc -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table table_desc select key, value from src POSTHOOK: type: QUERY POSTHOOK: Input: default@src diff --git ql/src/test/results/clientpositive/spark/groupby1.q.out ql/src/test/results/clientpositive/spark/groupby1.q.out index 847f45c..a337a39 100644 --- ql/src/test/results/clientpositive/spark/groupby1.q.out +++ ql/src/test/results/clientpositive/spark/groupby1.q.out @@ -100,7 +100,6 @@ PREHOOK: query: FROM src INSERT OVERWRITE TABLE dest_g1 SELECT src.key, sum(subs PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest_g1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM src INSERT OVERWRITE TABLE dest_g1 SELECT src.key, sum(substr(src.value,5)) GROUP BY src.key POSTHOOK: type: QUERY POSTHOOK: Input: default@src diff --git ql/src/test/results/clientpositive/spark/groupby10.q.out ql/src/test/results/clientpositive/spark/groupby10.q.out index 2095843..d2198de 100644 --- ql/src/test/results/clientpositive/spark/groupby10.q.out +++ ql/src/test/results/clientpositive/spark/groupby10.q.out @@ -177,8 +177,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@input PREHOOK: Output: default@dest1 PREHOOK: Output: default@dest2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM INPUT INSERT OVERWRITE TABLE dest1 SELECT INPUT.key, count(substr(INPUT.value,5)), count(distinct substr(INPUT.value,5)) GROUP BY INPUT.key INSERT OVERWRITE TABLE dest2 SELECT INPUT.key, sum(substr(INPUT.value,5)), sum(distinct substr(INPUT.value,5)) GROUP BY INPUT.key @@ -391,8 +389,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@input PREHOOK: Output: default@dest1 PREHOOK: Output: default@dest2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM INPUT INSERT OVERWRITE TABLE dest1 SELECT INPUT.key, count(substr(INPUT.value,5)), count(distinct substr(INPUT.value,5)) GROUP BY INPUT.key INSERT OVERWRITE TABLE dest2 SELECT INPUT.key, sum(substr(INPUT.value,5)), sum(distinct substr(INPUT.value,5)) GROUP BY INPUT.key @@ -607,8 +603,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@input PREHOOK: Output: default@dest1 PREHOOK: Output: default@dest2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM INPUT INSERT OVERWRITE TABLE dest1 SELECT INPUT.key, sum(distinct substr(INPUT.value,5)), count(distinct substr(INPUT.value,5)) GROUP BY INPUT.key INSERT OVERWRITE TABLE dest2 SELECT INPUT.key, sum(distinct substr(INPUT.value,5)), avg(distinct substr(INPUT.value,5)) GROUP BY INPUT.key diff --git ql/src/test/results/clientpositive/spark/groupby11.q.out ql/src/test/results/clientpositive/spark/groupby11.q.out index 70db5a5..730a4f3 100644 --- ql/src/test/results/clientpositive/spark/groupby11.q.out +++ ql/src/test/results/clientpositive/spark/groupby11.q.out @@ -171,8 +171,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1@ds=111 PREHOOK: Output: default@dest2@ds=111 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM src INSERT OVERWRITE TABLE dest1 partition(ds='111') SELECT src.value, count(src.key), count(distinct src.key) GROUP BY src.value diff --git ql/src/test/results/clientpositive/spark/groupby2.q.out ql/src/test/results/clientpositive/spark/groupby2.q.out index 86e2f2a..1408571 100644 --- ql/src/test/results/clientpositive/spark/groupby2.q.out +++ ql/src/test/results/clientpositive/spark/groupby2.q.out @@ -83,7 +83,6 @@ INSERT OVERWRITE TABLE dest_g2 SELECT substr(src.key,1,1), count(DISTINCT substr PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest_g2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM src INSERT OVERWRITE TABLE dest_g2 SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))) GROUP BY substr(src.key,1,1) POSTHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/spark/groupby3.q.out ql/src/test/results/clientpositive/spark/groupby3.q.out index 13a5fab..31c816f 100644 --- ql/src/test/results/clientpositive/spark/groupby3.q.out +++ ql/src/test/results/clientpositive/spark/groupby3.q.out @@ -121,7 +121,6 @@ INSERT OVERWRITE TABLE dest1 SELECT PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM src INSERT OVERWRITE TABLE dest1 SELECT sum(substr(src.value,5)), diff --git ql/src/test/results/clientpositive/spark/groupby3_map.q.out ql/src/test/results/clientpositive/spark/groupby3_map.q.out index dac2824..f8b5b88 100644 --- ql/src/test/results/clientpositive/spark/groupby3_map.q.out +++ ql/src/test/results/clientpositive/spark/groupby3_map.q.out @@ -115,7 +115,6 @@ INSERT OVERWRITE TABLE dest1 SELECT PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM src INSERT OVERWRITE TABLE dest1 SELECT sum(substr(src.value,5)), diff --git ql/src/test/results/clientpositive/spark/groupby3_map_multi_distinct.q.out ql/src/test/results/clientpositive/spark/groupby3_map_multi_distinct.q.out index d2c054a..03a80be 100644 --- ql/src/test/results/clientpositive/spark/groupby3_map_multi_distinct.q.out +++ ql/src/test/results/clientpositive/spark/groupby3_map_multi_distinct.q.out @@ -121,7 +121,6 @@ INSERT OVERWRITE TABLE dest1 SELECT PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM src INSERT OVERWRITE TABLE dest1 SELECT sum(substr(src.value,5)), diff --git ql/src/test/results/clientpositive/spark/groupby3_map_skew.q.out ql/src/test/results/clientpositive/spark/groupby3_map_skew.q.out index ec6439a..1778186 100644 --- ql/src/test/results/clientpositive/spark/groupby3_map_skew.q.out +++ ql/src/test/results/clientpositive/spark/groupby3_map_skew.q.out @@ -128,7 +128,6 @@ INSERT OVERWRITE TABLE dest1 SELECT PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM src INSERT OVERWRITE TABLE dest1 SELECT sum(substr(src.value,5)), diff --git ql/src/test/results/clientpositive/spark/groupby3_noskew.q.out ql/src/test/results/clientpositive/spark/groupby3_noskew.q.out index 0c9a7e1..2343bdd 100644 --- ql/src/test/results/clientpositive/spark/groupby3_noskew.q.out +++ ql/src/test/results/clientpositive/spark/groupby3_noskew.q.out @@ -108,7 +108,6 @@ INSERT OVERWRITE TABLE dest1 SELECT PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM src INSERT OVERWRITE TABLE dest1 SELECT sum(substr(src.value,5)), diff --git ql/src/test/results/clientpositive/spark/groupby3_noskew_multi_distinct.q.out ql/src/test/results/clientpositive/spark/groupby3_noskew_multi_distinct.q.out index 42fbb8c..bc51624 100644 --- ql/src/test/results/clientpositive/spark/groupby3_noskew_multi_distinct.q.out +++ ql/src/test/results/clientpositive/spark/groupby3_noskew_multi_distinct.q.out @@ -114,7 +114,6 @@ INSERT OVERWRITE TABLE dest1 SELECT PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM src INSERT OVERWRITE TABLE dest1 SELECT sum(substr(src.value,5)), diff --git ql/src/test/results/clientpositive/spark/groupby4.q.out ql/src/test/results/clientpositive/spark/groupby4.q.out index 318c5a3..38e4df7 100644 --- ql/src/test/results/clientpositive/spark/groupby4.q.out +++ ql/src/test/results/clientpositive/spark/groupby4.q.out @@ -99,7 +99,6 @@ INSERT OVERWRITE TABLE dest1 SELECT substr(src.key,1,1) GROUP BY substr(src.key, PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM src INSERT OVERWRITE TABLE dest1 SELECT substr(src.key,1,1) GROUP BY substr(src.key,1,1) POSTHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/spark/groupby7.q.out ql/src/test/results/clientpositive/spark/groupby7.q.out index 01af804..bb5f62c 100644 --- ql/src/test/results/clientpositive/spark/groupby7.q.out +++ ql/src/test/results/clientpositive/spark/groupby7.q.out @@ -25,8 +25,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 PREHOOK: Output: default@dest2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM SRC INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key diff --git ql/src/test/results/clientpositive/spark/groupby7_map.q.out ql/src/test/results/clientpositive/spark/groupby7_map.q.out index 22a05b5..9a3460f 100644 --- ql/src/test/results/clientpositive/spark/groupby7_map.q.out +++ ql/src/test/results/clientpositive/spark/groupby7_map.q.out @@ -158,8 +158,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 PREHOOK: Output: default@dest2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM SRC INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key diff --git ql/src/test/results/clientpositive/spark/groupby7_map_multi_single_reducer.q.out ql/src/test/results/clientpositive/spark/groupby7_map_multi_single_reducer.q.out index bc453c6..8dbbe19 100644 --- ql/src/test/results/clientpositive/spark/groupby7_map_multi_single_reducer.q.out +++ ql/src/test/results/clientpositive/spark/groupby7_map_multi_single_reducer.q.out @@ -135,8 +135,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 PREHOOK: Output: default@dest2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM SRC INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key diff --git ql/src/test/results/clientpositive/spark/groupby7_map_skew.q.out ql/src/test/results/clientpositive/spark/groupby7_map_skew.q.out index 2a07f2a..ed610cf 100644 --- ql/src/test/results/clientpositive/spark/groupby7_map_skew.q.out +++ ql/src/test/results/clientpositive/spark/groupby7_map_skew.q.out @@ -188,8 +188,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 PREHOOK: Output: default@dest2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM SRC INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key diff --git ql/src/test/results/clientpositive/spark/groupby7_noskew.q.out ql/src/test/results/clientpositive/spark/groupby7_noskew.q.out index 00a0707..ccfdf39 100644 --- ql/src/test/results/clientpositive/spark/groupby7_noskew.q.out +++ ql/src/test/results/clientpositive/spark/groupby7_noskew.q.out @@ -146,8 +146,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 PREHOOK: Output: default@dest2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM SRC INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key diff --git ql/src/test/results/clientpositive/spark/groupby7_noskew_multi_single_reducer.q.out ql/src/test/results/clientpositive/spark/groupby7_noskew_multi_single_reducer.q.out index 36640ef..9d94b0e 100644 --- ql/src/test/results/clientpositive/spark/groupby7_noskew_multi_single_reducer.q.out +++ ql/src/test/results/clientpositive/spark/groupby7_noskew_multi_single_reducer.q.out @@ -177,8 +177,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 PREHOOK: Output: default@dest2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM SRC INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key ORDER BY SRC.key limit 10 INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key ORDER BY SRC.key limit 10 diff --git ql/src/test/results/clientpositive/spark/groupby8.q.out ql/src/test/results/clientpositive/spark/groupby8.q.out index d8295ce..10cd709 100644 --- ql/src/test/results/clientpositive/spark/groupby8.q.out +++ ql/src/test/results/clientpositive/spark/groupby8.q.out @@ -161,8 +161,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 PREHOOK: Output: default@dest2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM SRC INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.key INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.key @@ -951,8 +949,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 PREHOOK: Output: default@dest2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM SRC INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.key INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.key diff --git ql/src/test/results/clientpositive/spark/groupby8_map.q.out ql/src/test/results/clientpositive/spark/groupby8_map.q.out index b9aa597..4d4152c 100644 --- ql/src/test/results/clientpositive/spark/groupby8_map.q.out +++ ql/src/test/results/clientpositive/spark/groupby8_map.q.out @@ -161,8 +161,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 PREHOOK: Output: default@dest2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM SRC INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.key INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.key diff --git ql/src/test/results/clientpositive/spark/groupby8_map_skew.q.out ql/src/test/results/clientpositive/spark/groupby8_map_skew.q.out index b9aa597..4d4152c 100644 --- ql/src/test/results/clientpositive/spark/groupby8_map_skew.q.out +++ ql/src/test/results/clientpositive/spark/groupby8_map_skew.q.out @@ -161,8 +161,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 PREHOOK: Output: default@dest2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM SRC INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.key INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.key diff --git ql/src/test/results/clientpositive/spark/groupby8_noskew.q.out ql/src/test/results/clientpositive/spark/groupby8_noskew.q.out index b9aa597..4d4152c 100644 --- ql/src/test/results/clientpositive/spark/groupby8_noskew.q.out +++ ql/src/test/results/clientpositive/spark/groupby8_noskew.q.out @@ -161,8 +161,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 PREHOOK: Output: default@dest2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM SRC INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.key INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.key diff --git ql/src/test/results/clientpositive/spark/groupby9.q.out ql/src/test/results/clientpositive/spark/groupby9.q.out index bec2346..8cb1e8a 100644 --- ql/src/test/results/clientpositive/spark/groupby9.q.out +++ ql/src/test/results/clientpositive/spark/groupby9.q.out @@ -161,8 +161,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 PREHOOK: Output: default@dest2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM SRC INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.key INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, SRC.value, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.key, SRC.value @@ -952,8 +950,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 PREHOOK: Output: default@dest2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM SRC INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.key INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, SRC.value, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.value, SRC.key @@ -1743,8 +1739,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 PREHOOK: Output: default@dest2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM SRC INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.key INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, SRC.value, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.key, SRC.value @@ -2531,8 +2525,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 PREHOOK: Output: default@dest2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM SRC INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, COUNT(SUBSTR(SRC.value,5)) GROUP BY SRC.key INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, SRC.value, COUNT(SUBSTR(SRC.value,5)) GROUP BY SRC.key, SRC.value @@ -3322,8 +3314,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 PREHOOK: Output: default@dest2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM SRC INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.key INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, SRC.value, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.value, SRC.key diff --git ql/src/test/results/clientpositive/spark/groupby_complex_types.q.out ql/src/test/results/clientpositive/spark/groupby_complex_types.q.out index 16fadea..7585064 100644 --- ql/src/test/results/clientpositive/spark/groupby_complex_types.q.out +++ ql/src/test/results/clientpositive/spark/groupby_complex_types.q.out @@ -222,9 +222,6 @@ PREHOOK: Input: default@src PREHOOK: Output: default@dest1 PREHOOK: Output: default@dest2 PREHOOK: Output: default@dest3 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM SRC INSERT OVERWRITE TABLE DEST1 SELECT ARRAY(SRC.key), COUNT(1) GROUP BY ARRAY(SRC.key) INSERT OVERWRITE TABLE DEST2 SELECT MAP(SRC.key, SRC.value), COUNT(1) GROUP BY MAP(SRC.key, SRC.value) diff --git ql/src/test/results/clientpositive/spark/groupby_complex_types_multi_single_reducer.q.out ql/src/test/results/clientpositive/spark/groupby_complex_types_multi_single_reducer.q.out index 7470843..d6314b9 100644 --- ql/src/test/results/clientpositive/spark/groupby_complex_types_multi_single_reducer.q.out +++ ql/src/test/results/clientpositive/spark/groupby_complex_types_multi_single_reducer.q.out @@ -192,8 +192,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 PREHOOK: Output: default@dest2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM SRC INSERT OVERWRITE TABLE DEST1 SELECT ARRAY(SRC.key), COUNT(1) GROUP BY ARRAY(SRC.key) limit 10 INSERT OVERWRITE TABLE DEST2 SELECT MAP(SRC.key, SRC.value), COUNT(1) GROUP BY MAP(SRC.key, SRC.value) limit 10 diff --git ql/src/test/results/clientpositive/spark/groupby_cube1.q.out ql/src/test/results/clientpositive/spark/groupby_cube1.q.out index 169c4ac..01a6b48 100644 --- ql/src/test/results/clientpositive/spark/groupby_cube1.q.out +++ ql/src/test/results/clientpositive/spark/groupby_cube1.q.out @@ -553,8 +553,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@t1 PREHOOK: Output: default@t2 PREHOOK: Output: default@t3 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM T1 INSERT OVERWRITE TABLE T2 SELECT key, val, count(1) group by key, val with cube INSERT OVERWRITE TABLE T3 SELECT key, val, sum(1) group by key, val with cube diff --git ql/src/test/results/clientpositive/spark/groupby_multi_insert_common_distinct.q.out ql/src/test/results/clientpositive/spark/groupby_multi_insert_common_distinct.q.out index d3457da..5f7b87a 100644 --- ql/src/test/results/clientpositive/spark/groupby_multi_insert_common_distinct.q.out +++ ql/src/test/results/clientpositive/spark/groupby_multi_insert_common_distinct.q.out @@ -161,8 +161,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 PREHOOK: Output: default@dest2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: from src insert overwrite table dest1 select key, count(distinct value) group by key insert overwrite table dest2 select key+key, count(distinct value) group by key+key @@ -342,8 +340,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 PREHOOK: Output: default@dest2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: from src insert overwrite table dest1 select key, count(distinct value) group by key insert overwrite table dest2 select key+key, count(distinct value) group by key+key diff --git ql/src/test/results/clientpositive/spark/groupby_multi_single_reducer.q.out ql/src/test/results/clientpositive/spark/groupby_multi_single_reducer.q.out index 3abd0e3..df10b08 100644 --- ql/src/test/results/clientpositive/spark/groupby_multi_single_reducer.q.out +++ ql/src/test/results/clientpositive/spark/groupby_multi_single_reducer.q.out @@ -202,9 +202,6 @@ PREHOOK: Input: default@src PREHOOK: Output: default@dest_g2 PREHOOK: Output: default@dest_g3 PREHOOK: Output: default@dest_g4 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM src INSERT OVERWRITE TABLE dest_g2 SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))), sum(DISTINCT substr(src.value, 5)), count(src.value) WHERE substr(src.key,1,1) >= 5 GROUP BY substr(src.key,1,1) INSERT OVERWRITE TABLE dest_g3 SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))), sum(DISTINCT substr(src.value, 5)), count(src.value) WHERE substr(src.key,1,1) < 5 GROUP BY substr(src.key,1,1) @@ -546,11 +543,6 @@ PREHOOK: Output: default@dest_g3 PREHOOK: Output: default@dest_g4 PREHOOK: Output: default@dest_h2 PREHOOK: Output: default@dest_h3 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM src INSERT OVERWRITE TABLE dest_g2 SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))), sum(DISTINCT substr(src.value, 5)), count(src.value) WHERE substr(src.key,1,1) >= 5 GROUP BY substr(src.key,1,1) INSERT OVERWRITE TABLE dest_g3 SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))), sum(DISTINCT substr(src.value, 5)), count(src.value) WHERE substr(src.key,1,1) < 5 GROUP BY substr(src.key,1,1) diff --git ql/src/test/results/clientpositive/spark/groupby_multi_single_reducer2.q.out ql/src/test/results/clientpositive/spark/groupby_multi_single_reducer2.q.out index 7f74c62..c944e19 100644 --- ql/src/test/results/clientpositive/spark/groupby_multi_single_reducer2.q.out +++ ql/src/test/results/clientpositive/spark/groupby_multi_single_reducer2.q.out @@ -140,8 +140,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest_g2 PREHOOK: Output: default@dest_g3 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM src INSERT OVERWRITE TABLE dest_g2 SELECT substr(src.key,1,1), count(DISTINCT src.key) WHERE substr(src.key,1,1) >= 5 GROUP BY substr(src.key,1,1) INSERT OVERWRITE TABLE dest_g3 SELECT substr(src.key,1,1), count(DISTINCT src.key), count(src.value) WHERE substr(src.key,1,1) < 5 GROUP BY substr(src.key,1,1) diff --git ql/src/test/results/clientpositive/spark/groupby_multi_single_reducer3.q.out ql/src/test/results/clientpositive/spark/groupby_multi_single_reducer3.q.out index c4b7419..a137c46 100644 --- ql/src/test/results/clientpositive/spark/groupby_multi_single_reducer3.q.out +++ ql/src/test/results/clientpositive/spark/groupby_multi_single_reducer3.q.out @@ -160,8 +160,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@e1 PREHOOK: Output: default@e2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: from src insert overwrite table e1 select key, count(*) @@ -342,8 +340,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@e1 PREHOOK: Output: default@e2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: from src insert overwrite table e1 select value, count(*) @@ -524,8 +520,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@e1 PREHOOK: Output: default@e2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: from src insert overwrite table e1 select key, count(*) @@ -706,8 +700,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@e1 PREHOOK: Output: default@e2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: from src insert overwrite table e1 select value, count(*) diff --git ql/src/test/results/clientpositive/spark/groupby_position.q.out ql/src/test/results/clientpositive/spark/groupby_position.q.out index 9e58189..c6fd342 100644 --- ql/src/test/results/clientpositive/spark/groupby_position.q.out +++ ql/src/test/results/clientpositive/spark/groupby_position.q.out @@ -162,8 +162,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@testtable1 PREHOOK: Output: default@testtable2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM SRC INSERT OVERWRITE TABLE testTable1 SELECT SRC.key, COUNT(DISTINCT SUBSTR(SRC.value,5)) WHERE SRC.key < 20 GROUP BY 1 INSERT OVERWRITE TABLE testTable2 SELECT SRC.key, SRC.value, COUNT(DISTINCT SUBSTR(SRC.value,5)) WHERE SRC.key < 20 GROUP BY 1, 2 @@ -362,8 +360,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@testtable1 PREHOOK: Output: default@testtable2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM SRC INSERT OVERWRITE TABLE testTable1 SELECT SRC.key, COUNT(DISTINCT SUBSTR(SRC.value,5)) WHERE SRC.key < 20 GROUP BY 1 INSERT OVERWRITE TABLE testTable2 SELECT SRC.key, SRC.value, COUNT(DISTINCT SUBSTR(SRC.value,5)) WHERE SRC.key < 20 GROUP BY 2, 1 diff --git ql/src/test/results/clientpositive/spark/groupby_ppr.q.out ql/src/test/results/clientpositive/spark/groupby_ppr.q.out index 860aa58..65585b1 100644 --- ql/src/test/results/clientpositive/spark/groupby_ppr.q.out +++ ql/src/test/results/clientpositive/spark/groupby_ppr.q.out @@ -298,7 +298,6 @@ PREHOOK: Input: default@srcpart PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 PREHOOK: Output: default@dest1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM srcpart src INSERT OVERWRITE TABLE dest1 SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))) diff --git ql/src/test/results/clientpositive/spark/groupby_rollup1.q.out ql/src/test/results/clientpositive/spark/groupby_rollup1.q.out index 0aeff6b..4af8f38 100644 --- ql/src/test/results/clientpositive/spark/groupby_rollup1.q.out +++ ql/src/test/results/clientpositive/spark/groupby_rollup1.q.out @@ -541,8 +541,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@t1 PREHOOK: Output: default@t2 PREHOOK: Output: default@t3 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM T1 INSERT OVERWRITE TABLE T2 SELECT key, val, count(1) group by key, val with rollup INSERT OVERWRITE TABLE T3 SELECT key, val, sum(1) group by key, val with rollup diff --git ql/src/test/results/clientpositive/spark/groupby_sort_1_23.q.out ql/src/test/results/clientpositive/spark/groupby_sort_1_23.q.out index 61dd2be..fdf84db 100644 --- ql/src/test/results/clientpositive/spark/groupby_sort_1_23.q.out +++ ql/src/test/results/clientpositive/spark/groupby_sort_1_23.q.out @@ -27,7 +27,6 @@ INSERT OVERWRITE TABLE T1 select key, val from T1 PREHOOK: type: QUERY PREHOOK: Input: default@t1 PREHOOK: Output: default@t1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: -- perform an insert to make sure there are 2 files INSERT OVERWRITE TABLE T1 select key, val from T1 POSTHOOK: type: QUERY @@ -95,28 +94,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t1 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE GatherStats: false Select Operator expressions: key (type: string) outputColumnNames: key - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) keys: key (type: string) mode: final outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: UDFToInteger(_col0) (type: int), UDFToInteger(_col1) (type: int) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 1 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -156,8 +155,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -178,8 +177,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -225,7 +224,6 @@ SELECT key, count(1) FROM T1 GROUP BY key PREHOOK: type: QUERY PREHOOK: Input: default@t1 PREHOOK: Output: default@outputtbl1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE outputTbl1 SELECT key, count(1) FROM T1 GROUP BY key POSTHOOK: type: QUERY @@ -311,23 +309,23 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t1 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE GatherStats: false Select Operator expressions: key (type: string), val (type: string) outputColumnNames: key, val - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) keys: key (type: string), val (type: string) mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string), _col1 (type: string) sort order: ++ Map-reduce partition columns: _col0 (type: string), _col1 (type: string) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col2 (type: bigint) auto parallelism: true @@ -350,8 +348,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -372,8 +370,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -392,17 +390,17 @@ STAGE PLANS: keys: KEY._col0 (type: string), KEY._col1 (type: string) mode: mergepartial outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: UDFToInteger(_col0) (type: int), _col1 (type: string), UDFToInteger(_col2) (type: int) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 1 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -458,7 +456,6 @@ SELECT key, val, count(1) FROM T1 GROUP BY key, val PREHOOK: type: QUERY PREHOOK: Input: default@t1 PREHOOK: Output: default@outputtbl2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE outputTbl2 SELECT key, val, count(1) FROM T1 GROUP BY key, val POSTHOOK: type: QUERY @@ -546,28 +543,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t1 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE GatherStats: false Select Operator expressions: key (type: string) outputColumnNames: _col0 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) keys: _col0 (type: string) mode: final outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: UDFToInteger(_col0) (type: int), UDFToInteger(_col1) (type: int) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 1 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -581,8 +578,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 5 + rawDataSize 15 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -612,8 +609,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -634,8 +631,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -667,8 +664,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 5 + rawDataSize 15 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -686,7 +683,6 @@ SELECT key, count(1) FROM (SELECT key, val FROM T1) subq1 GROUP BY key PREHOOK: type: QUERY PREHOOK: Input: default@t1 PREHOOK: Output: default@outputtbl1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE outputTbl1 SELECT key, count(1) FROM (SELECT key, val FROM T1) subq1 GROUP BY key POSTHOOK: type: QUERY @@ -774,28 +770,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t1 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE GatherStats: false Select Operator expressions: key (type: string) outputColumnNames: _col0 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) keys: _col0 (type: string) mode: final outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: UDFToInteger(_col0) (type: int), UDFToInteger(_col1) (type: int) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 1 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -809,8 +805,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 5 + rawDataSize 15 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -840,8 +836,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -862,8 +858,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -895,8 +891,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 5 + rawDataSize 15 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -914,7 +910,6 @@ SELECT k, count(1) FROM (SELECT key as k, val as v FROM T1) subq1 GROUP BY k PREHOOK: type: QUERY PREHOOK: Input: default@t1 PREHOOK: Output: default@outputtbl1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE outputTbl1 SELECT k, count(1) FROM (SELECT key as k, val as v FROM T1) subq1 GROUP BY k POSTHOOK: type: QUERY @@ -998,28 +993,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t1 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE GatherStats: false Select Operator expressions: key (type: string) outputColumnNames: key - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) keys: 1 (type: int), key (type: string) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), UDFToInteger(_col1) (type: int), UDFToInteger(_col2) (type: int) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 1 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -1059,8 +1054,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1081,8 +1076,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1128,7 +1123,6 @@ SELECT 1, key, count(1) FROM T1 GROUP BY 1, key PREHOOK: type: QUERY PREHOOK: Input: default@t1 PREHOOK: Output: default@outputtbl3 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE outputTbl3 SELECT 1, key, count(1) FROM T1 GROUP BY 1, key POSTHOOK: type: QUERY @@ -1218,23 +1212,23 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t1 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE GatherStats: false Select Operator expressions: key (type: string), val (type: string) outputColumnNames: key, val - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) keys: key (type: string), 1 (type: int), val (type: string) mode: hash outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string), _col1 (type: int), _col2 (type: string) sort order: +++ Map-reduce partition columns: _col0 (type: string), _col1 (type: int), _col2 (type: string) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col3 (type: bigint) auto parallelism: true @@ -1257,8 +1251,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1279,8 +1273,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1299,17 +1293,17 @@ STAGE PLANS: keys: KEY._col0 (type: string), KEY._col1 (type: int), KEY._col2 (type: string) mode: mergepartial outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: UDFToInteger(_col0) (type: int), _col1 (type: int), _col2 (type: string), UDFToInteger(_col3) (type: int) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 1 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -1365,7 +1359,6 @@ SELECT key, 1, val, count(1) FROM T1 GROUP BY key, 1, val PREHOOK: type: QUERY PREHOOK: Input: default@t1 PREHOOK: Output: default@outputtbl4 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE outputTbl4 SELECT key, 1, val, count(1) FROM T1 GROUP BY key, 1, val POSTHOOK: type: QUERY @@ -1450,23 +1443,23 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t1 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE GatherStats: false Select Operator expressions: key (type: string) outputColumnNames: key - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) keys: key (type: string), (key + 1) (type: double) mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string), _col1 (type: double) sort order: ++ Map-reduce partition columns: _col0 (type: string), _col1 (type: double) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col2 (type: bigint) auto parallelism: true @@ -1489,8 +1482,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1511,8 +1504,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1531,17 +1524,17 @@ STAGE PLANS: keys: KEY._col0 (type: string), KEY._col1 (type: double) mode: mergepartial outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: UDFToInteger(_col0) (type: int), UDFToInteger(_col1) (type: int), UDFToInteger(_col2) (type: int) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 1 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -1555,8 +1548,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl3 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 5 + rawDataSize 25 serialization.ddl struct outputtbl3 { i32 key1, i32 key2, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1588,8 +1581,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl3 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 5 + rawDataSize 25 serialization.ddl struct outputtbl3 { i32 key1, i32 key2, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1607,7 +1600,6 @@ SELECT key, key + 1, count(1) FROM T1 GROUP BY key, key + 1 PREHOOK: type: QUERY PREHOOK: Input: default@t1 PREHOOK: Output: default@outputtbl3 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE outputTbl3 SELECT key, key + 1, count(1) FROM T1 GROUP BY key, key + 1 POSTHOOK: type: QUERY @@ -1718,33 +1710,33 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t1 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE GatherStats: false Select Operator expressions: key (type: string) outputColumnNames: key - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) keys: key (type: string) mode: final outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col1 (type: bigint) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: sum(_col1) keys: (_col0 + _col0) (type: double) mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: double) sort order: + Map-reduce partition columns: _col0 (type: double) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: bigint) auto parallelism: true @@ -1767,8 +1759,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1789,8 +1781,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1809,17 +1801,17 @@ STAGE PLANS: keys: KEY._col0 (type: double) mode: mergepartial outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: UDFToInteger(_col0) (type: int), UDFToInteger(_col1) (type: int) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 1 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -1833,8 +1825,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 5 + rawDataSize 15 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1866,8 +1858,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 5 + rawDataSize 15 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1887,7 +1879,6 @@ group by key + key PREHOOK: type: QUERY PREHOOK: Input: default@t1 PREHOOK: Output: default@outputtbl1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE outputTbl1 SELECT key + key, sum(cnt) from (SELECT key, count(1) as cnt FROM T1 GROUP BY key) subq1 @@ -2035,8 +2026,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 5 + rawDataSize 17 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2066,8 +2057,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2088,8 +2079,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2137,8 +2128,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 5 + rawDataSize 17 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2168,8 +2159,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2190,8 +2181,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2225,8 +2216,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 5 + rawDataSize 17 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2248,7 +2239,6 @@ SELECT key, count(1) FROM T1 GROUP BY key PREHOOK: type: QUERY PREHOOK: Input: default@t1 PREHOOK: Output: default@outputtbl1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE outputTbl1 SELECT * FROM ( SELECT key, count(1) FROM T1 GROUP BY key @@ -2411,8 +2401,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl1 numFiles 2 - numRows -1 - rawDataSize -1 + numRows 0 + rawDataSize 0 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2442,8 +2432,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2464,8 +2454,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2480,23 +2470,23 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t1 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE GatherStats: false Select Operator expressions: key (type: string) outputColumnNames: key - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) keys: (key + key) (type: double) mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: double) sort order: + Map-reduce partition columns: _col0 (type: double) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: bigint) auto parallelism: true @@ -2519,8 +2509,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2541,8 +2531,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2585,8 +2575,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl1 numFiles 2 - numRows -1 - rawDataSize -1 + numRows 0 + rawDataSize 0 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2620,8 +2610,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl1 numFiles 2 - numRows -1 - rawDataSize -1 + numRows 0 + rawDataSize 0 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2643,7 +2633,6 @@ SELECT key + key as key, count(1) as cnt FROM T1 GROUP BY key + key PREHOOK: type: QUERY PREHOOK: Input: default@t1 PREHOOK: Output: default@outputtbl1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE outputTbl1 SELECT * FROM ( SELECT key, count(1) as cnt FROM T1 GROUP BY key @@ -2791,31 +2780,31 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t1 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: key is not null (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string) outputColumnNames: key - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) keys: key (type: string) mode: final outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col1 (type: bigint) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) sort order: + Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE tag: 0 value expressions: _col1 (type: bigint) auto parallelism: true @@ -2838,8 +2827,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2860,8 +2849,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2876,31 +2865,31 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t1 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: key is not null (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string) outputColumnNames: key - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) keys: key (type: string) mode: final outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col1 (type: bigint) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) sort order: + Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE tag: 1 value expressions: _col1 (type: bigint) auto parallelism: true @@ -2923,8 +2912,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2945,8 +2934,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2967,17 +2956,17 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 {VALUE._col0} outputColumnNames: _col0, _col1, _col3 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: UDFToInteger(_col0) (type: int), UDFToInteger((_col1 + _col3)) (type: int) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 1 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -2991,8 +2980,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl1 numFiles 2 - numRows -1 - rawDataSize -1 + numRows 0 + rawDataSize 0 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -3024,8 +3013,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl1 numFiles 2 - numRows -1 - rawDataSize -1 + numRows 0 + rawDataSize 0 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -3047,7 +3036,6 @@ ON subq1.key = subq2.key PREHOOK: type: QUERY PREHOOK: Input: default@t1 PREHOOK: Output: default@outputtbl1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE outputTbl1 SELECT subq1.key, subq1.cnt+subq2.cnt FROM (SELECT key, count(1) as cnt FROM T1 GROUP BY key) subq1 @@ -3176,31 +3164,31 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t1 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: key is not null (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string) outputColumnNames: key - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) keys: key (type: string) mode: final outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col1 (type: bigint) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) sort order: + Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE tag: 0 value expressions: _col1 (type: bigint) auto parallelism: true @@ -3223,8 +3211,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -3245,8 +3233,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -3261,27 +3249,27 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t1 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: key is not null (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string), val (type: string) outputColumnNames: key, val - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) keys: key (type: string), val (type: string) mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string), _col1 (type: string) sort order: ++ Map-reduce partition columns: _col0 (type: string), _col1 (type: string) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col2 (type: bigint) auto parallelism: true @@ -3304,8 +3292,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -3326,8 +3314,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -3348,17 +3336,17 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 {KEY.reducesinkkey0} {VALUE._col0} {VALUE._col1} outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col1 (type: bigint), _col2 (type: string), _col3 (type: string), _col4 (type: bigint) outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -3382,16 +3370,16 @@ STAGE PLANS: keys: KEY._col0 (type: string), KEY._col1 (type: string) mode: mergepartial outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col1 (type: string), _col2 (type: bigint) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) sort order: + Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE tag: 1 value expressions: _col1 (type: string), _col2 (type: bigint) auto parallelism: true @@ -3417,7 +3405,6 @@ INSERT OVERWRITE TABLE T2 select key, val from T1 PREHOOK: type: QUERY PREHOOK: Input: default@t1 PREHOOK: Output: default@t2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: -- perform an insert to make sure there are 2 files INSERT OVERWRITE TABLE T2 select key, val from T1 POSTHOOK: type: QUERY @@ -3477,24 +3464,24 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t2 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE GatherStats: false Select Operator expressions: key (type: string) outputColumnNames: key - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) bucketGroup: true keys: key (type: string) mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) sort order: + Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: bigint) auto parallelism: true @@ -3517,8 +3504,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t2 numFiles 2 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t2 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -3539,8 +3526,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t2 numFiles 2 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t2 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -3559,17 +3546,17 @@ STAGE PLANS: keys: KEY._col0 (type: string) mode: mergepartial outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: UDFToInteger(_col0) (type: int), UDFToInteger(_col1) (type: int) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 1 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -3583,8 +3570,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 5 + rawDataSize 15 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -3616,8 +3603,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 5 + rawDataSize 15 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -3635,7 +3622,6 @@ SELECT key, count(1) FROM T2 GROUP BY key PREHOOK: type: QUERY PREHOOK: Input: default@t2 PREHOOK: Output: default@outputtbl1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE outputTbl1 SELECT key, count(1) FROM T2 GROUP BY key POSTHOOK: type: QUERY @@ -3716,28 +3702,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t2 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE GatherStats: false Select Operator expressions: key (type: string), val (type: string) outputColumnNames: key, val - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) keys: key (type: string), 1 (type: int), val (type: string) mode: final outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: UDFToInteger(_col0) (type: int), _col1 (type: int), _col2 (type: string), UDFToInteger(_col3) (type: int) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 1 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -3751,8 +3737,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl4 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 48 serialization.ddl struct outputtbl4 { i32 key1, i32 key2, string key3, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -3782,8 +3768,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t2 numFiles 2 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t2 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -3804,8 +3790,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t2 numFiles 2 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t2 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -3837,8 +3823,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl4 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 48 serialization.ddl struct outputtbl4 { i32 key1, i32 key2, string key3, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -3856,7 +3842,6 @@ SELECT key, 1, val, count(1) FROM T2 GROUP BY key, 1, val PREHOOK: type: QUERY PREHOOK: Input: default@t2 PREHOOK: Output: default@outputtbl4 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE outputTbl4 SELECT key, 1, val, count(1) FROM T2 GROUP BY key, 1, val POSTHOOK: type: QUERY @@ -3951,28 +3936,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t2 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE GatherStats: false Select Operator expressions: key (type: string), val (type: string) outputColumnNames: key, val - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) keys: key (type: string), 1 (type: int), val (type: string), 2 (type: int) mode: final outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: UDFToInteger(_col0) (type: int), _col1 (type: int), _col2 (type: string), _col3 (type: int), UDFToInteger(_col4) (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 1 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -4012,8 +3997,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t2 numFiles 2 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t2 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -4034,8 +4019,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t2 numFiles 2 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t2 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -4081,7 +4066,6 @@ SELECT key, 1, val, 2, count(1) FROM T2 GROUP BY key, 1, val, 2 PREHOOK: type: QUERY PREHOOK: Input: default@t2 PREHOOK: Output: default@outputtbl5 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE outputTbl5 SELECT key, 1, val, 2, count(1) FROM T2 GROUP BY key, 1, val, 2 POSTHOOK: type: QUERY @@ -4190,28 +4174,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t2 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE GatherStats: false Select Operator expressions: key (type: string), 1 (type: int), val (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) keys: _col0 (type: string), _col1 (type: int), _col2 (type: string) mode: final outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: UDFToInteger(_col0) (type: int), _col1 (type: int), _col2 (type: string), UDFToInteger(_col3) (type: int) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 1 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -4225,8 +4209,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl4 numFiles 2 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 48 serialization.ddl struct outputtbl4 { i32 key1, i32 key2, string key3, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -4256,8 +4240,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t2 numFiles 2 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t2 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -4278,8 +4262,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t2 numFiles 2 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t2 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -4311,8 +4295,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl4 numFiles 2 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 48 serialization.ddl struct outputtbl4 { i32 key1, i32 key2, string key3, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -4332,7 +4316,6 @@ group by key, constant, val PREHOOK: type: QUERY PREHOOK: Input: default@t2 PREHOOK: Output: default@outputtbl4 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE outputTbl4 SELECT key, constant, val, count(1) from (SELECT key, 1 as constant, val from T2)subq @@ -4468,28 +4451,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t2 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE GatherStats: false Select Operator expressions: key (type: string), 2 (type: int), val (type: string) outputColumnNames: _col0, _col3, _col2 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) keys: _col0 (type: string), _col3 (type: int), _col2 (type: string) mode: final outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: UDFToInteger(_col0) (type: int), _col1 (type: int), _col2 (type: string), UDFToInteger(_col3) (type: int) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 1 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -4503,8 +4486,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl4 numFiles 2 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 48 serialization.ddl struct outputtbl4 { i32 key1, i32 key2, string key3, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -4534,8 +4517,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t2 numFiles 2 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t2 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -4556,8 +4539,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t2 numFiles 2 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t2 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -4589,8 +4572,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl4 numFiles 2 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 48 serialization.ddl struct outputtbl4 { i32 key1, i32 key2, string key3, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -4613,7 +4596,6 @@ group by key, constant3, val PREHOOK: type: QUERY PREHOOK: Input: default@t2 PREHOOK: Output: default@outputtbl4 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE outputTbl4 select key, constant3, val, count(1) from ( @@ -4687,41 +4669,41 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t2 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string) outputColumnNames: key - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) bucketGroup: true keys: key (type: string) mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) sort order: + Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Select Operator expressions: key (type: string), val (type: string) outputColumnNames: key, val - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) keys: key (type: string), val (type: string) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: UDFToInteger(_col0) (type: int), _col1 (type: string), UDFToInteger(_col2) (type: int) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: true - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -4734,14 +4716,14 @@ STAGE PLANS: keys: KEY._col0 (type: string) mode: mergepartial outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: UDFToInteger(_col0) (type: int), UDFToInteger(_col1) (type: int) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: true - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -4784,8 +4766,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@t2 PREHOOK: Output: default@dest1 PREHOOK: Output: default@dest2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM T2 INSERT OVERWRITE TABLE DEST1 SELECT key, count(1) GROUP BY key INSERT OVERWRITE TABLE DEST2 SELECT key, val, count(1) GROUP BY key, val @@ -4856,48 +4836,48 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t2 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (key = 8) (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: val (type: string) outputColumnNames: _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: '8' (type: string) outputColumnNames: _col0 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) bucketGroup: true keys: _col0 (type: string) mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) sort order: + Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Select Operator expressions: '8' (type: string), _col1 (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) keys: _col0 (type: string), _col1 (type: string) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: UDFToInteger(_col0) (type: int), _col1 (type: string), UDFToInteger(_col2) (type: int) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: true - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -4910,14 +4890,14 @@ STAGE PLANS: keys: KEY._col0 (type: string) mode: mergepartial outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: UDFToInteger(_col0) (type: int), UDFToInteger(_col1) (type: int) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: true - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -4960,8 +4940,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@t2 PREHOOK: Output: default@dest1 PREHOOK: Output: default@dest2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM (select key, val from T2 where key = 8) x INSERT OVERWRITE TABLE DEST1 SELECT key, count(1) GROUP BY key INSERT OVERWRITE TABLE DEST2 SELECT key, val, count(1) GROUP BY key, val diff --git ql/src/test/results/clientpositive/spark/groupby_sort_skew_1_23.q.out ql/src/test/results/clientpositive/spark/groupby_sort_skew_1_23.q.out index 99da734..553b390 100644 --- ql/src/test/results/clientpositive/spark/groupby_sort_skew_1_23.q.out +++ ql/src/test/results/clientpositive/spark/groupby_sort_skew_1_23.q.out @@ -27,7 +27,6 @@ INSERT OVERWRITE TABLE T1 select key, val from T1 PREHOOK: type: QUERY PREHOOK: Input: default@t1 PREHOOK: Output: default@t1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: -- perform an insert to make sure there are 2 files INSERT OVERWRITE TABLE T1 select key, val from T1 POSTHOOK: type: QUERY @@ -95,28 +94,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t1 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE GatherStats: false Select Operator expressions: key (type: string) outputColumnNames: key - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) keys: key (type: string) mode: final outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: UDFToInteger(_col0) (type: int), UDFToInteger(_col1) (type: int) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 1 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -156,8 +155,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -178,8 +177,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -225,7 +224,6 @@ SELECT key, count(1) FROM T1 GROUP BY key PREHOOK: type: QUERY PREHOOK: Input: default@t1 PREHOOK: Output: default@outputtbl1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE outputTbl1 SELECT key, count(1) FROM T1 GROUP BY key POSTHOOK: type: QUERY @@ -312,23 +310,23 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t1 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE GatherStats: false Select Operator expressions: key (type: string), val (type: string) outputColumnNames: key, val - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) keys: key (type: string), val (type: string) mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string), _col1 (type: string) sort order: ++ Map-reduce partition columns: rand() (type: double) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col2 (type: bigint) auto parallelism: true @@ -351,8 +349,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -373,8 +371,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -393,12 +391,12 @@ STAGE PLANS: keys: KEY._col0 (type: string), KEY._col1 (type: string) mode: partials outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string), _col1 (type: string) sort order: ++ Map-reduce partition columns: _col0 (type: string), _col1 (type: string) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col2 (type: bigint) auto parallelism: true @@ -410,17 +408,17 @@ STAGE PLANS: keys: KEY._col0 (type: string), KEY._col1 (type: string) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: UDFToInteger(_col0) (type: int), _col1 (type: string), UDFToInteger(_col2) (type: int) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 1 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -476,7 +474,6 @@ SELECT key, val, count(1) FROM T1 GROUP BY key, val PREHOOK: type: QUERY PREHOOK: Input: default@t1 PREHOOK: Output: default@outputtbl2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE outputTbl2 SELECT key, val, count(1) FROM T1 GROUP BY key, val POSTHOOK: type: QUERY @@ -564,28 +561,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t1 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE GatherStats: false Select Operator expressions: key (type: string) outputColumnNames: _col0 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) keys: _col0 (type: string) mode: final outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: UDFToInteger(_col0) (type: int), UDFToInteger(_col1) (type: int) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 1 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -599,8 +596,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 5 + rawDataSize 15 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -630,8 +627,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -652,8 +649,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -685,8 +682,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 5 + rawDataSize 15 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -704,7 +701,6 @@ SELECT key, count(1) FROM (SELECT key, val FROM T1) subq1 GROUP BY key PREHOOK: type: QUERY PREHOOK: Input: default@t1 PREHOOK: Output: default@outputtbl1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE outputTbl1 SELECT key, count(1) FROM (SELECT key, val FROM T1) subq1 GROUP BY key POSTHOOK: type: QUERY @@ -792,28 +788,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t1 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE GatherStats: false Select Operator expressions: key (type: string) outputColumnNames: _col0 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) keys: _col0 (type: string) mode: final outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: UDFToInteger(_col0) (type: int), UDFToInteger(_col1) (type: int) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 1 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -827,8 +823,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 5 + rawDataSize 15 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -858,8 +854,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -880,8 +876,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -913,8 +909,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 5 + rawDataSize 15 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -932,7 +928,6 @@ SELECT k, count(1) FROM (SELECT key as k, val as v FROM T1) subq1 GROUP BY k PREHOOK: type: QUERY PREHOOK: Input: default@t1 PREHOOK: Output: default@outputtbl1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE outputTbl1 SELECT k, count(1) FROM (SELECT key as k, val as v FROM T1) subq1 GROUP BY k POSTHOOK: type: QUERY @@ -1016,28 +1011,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t1 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE GatherStats: false Select Operator expressions: key (type: string) outputColumnNames: key - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) keys: 1 (type: int), key (type: string) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), UDFToInteger(_col1) (type: int), UDFToInteger(_col2) (type: int) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 1 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -1077,8 +1072,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1099,8 +1094,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1146,7 +1141,6 @@ SELECT 1, key, count(1) FROM T1 GROUP BY 1, key PREHOOK: type: QUERY PREHOOK: Input: default@t1 PREHOOK: Output: default@outputtbl3 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE outputTbl3 SELECT 1, key, count(1) FROM T1 GROUP BY 1, key POSTHOOK: type: QUERY @@ -1237,23 +1231,23 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t1 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE GatherStats: false Select Operator expressions: key (type: string), val (type: string) outputColumnNames: key, val - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) keys: key (type: string), 1 (type: int), val (type: string) mode: hash outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string), _col1 (type: int), _col2 (type: string) sort order: +++ Map-reduce partition columns: rand() (type: double) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col3 (type: bigint) auto parallelism: true @@ -1276,8 +1270,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1298,8 +1292,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1318,12 +1312,12 @@ STAGE PLANS: keys: KEY._col0 (type: string), KEY._col1 (type: int), KEY._col2 (type: string) mode: partials outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string), _col1 (type: int), _col2 (type: string) sort order: +++ Map-reduce partition columns: _col0 (type: string), _col1 (type: int), _col2 (type: string) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col3 (type: bigint) auto parallelism: true @@ -1335,17 +1329,17 @@ STAGE PLANS: keys: KEY._col0 (type: string), KEY._col1 (type: int), KEY._col2 (type: string) mode: final outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: UDFToInteger(_col0) (type: int), _col1 (type: int), _col2 (type: string), UDFToInteger(_col3) (type: int) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 1 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -1401,7 +1395,6 @@ SELECT key, 1, val, count(1) FROM T1 GROUP BY key, 1, val PREHOOK: type: QUERY PREHOOK: Input: default@t1 PREHOOK: Output: default@outputtbl4 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE outputTbl4 SELECT key, 1, val, count(1) FROM T1 GROUP BY key, 1, val POSTHOOK: type: QUERY @@ -1487,23 +1480,23 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t1 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE GatherStats: false Select Operator expressions: key (type: string) outputColumnNames: key - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) keys: key (type: string), (key + 1) (type: double) mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string), _col1 (type: double) sort order: ++ Map-reduce partition columns: rand() (type: double) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col2 (type: bigint) auto parallelism: true @@ -1526,8 +1519,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1548,8 +1541,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1568,12 +1561,12 @@ STAGE PLANS: keys: KEY._col0 (type: string), KEY._col1 (type: double) mode: partials outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string), _col1 (type: double) sort order: ++ Map-reduce partition columns: _col0 (type: string), _col1 (type: double) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col2 (type: bigint) auto parallelism: true @@ -1585,17 +1578,17 @@ STAGE PLANS: keys: KEY._col0 (type: string), KEY._col1 (type: double) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: UDFToInteger(_col0) (type: int), UDFToInteger(_col1) (type: int), UDFToInteger(_col2) (type: int) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 1 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -1609,8 +1602,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl3 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 5 + rawDataSize 25 serialization.ddl struct outputtbl3 { i32 key1, i32 key2, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1642,8 +1635,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl3 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 5 + rawDataSize 25 serialization.ddl struct outputtbl3 { i32 key1, i32 key2, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1661,7 +1654,6 @@ SELECT key, key + 1, count(1) FROM T1 GROUP BY key, key + 1 PREHOOK: type: QUERY PREHOOK: Input: default@t1 PREHOOK: Output: default@outputtbl3 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE outputTbl3 SELECT key, key + 1, count(1) FROM T1 GROUP BY key, key + 1 POSTHOOK: type: QUERY @@ -1773,33 +1765,33 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t1 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE GatherStats: false Select Operator expressions: key (type: string) outputColumnNames: key - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) keys: key (type: string) mode: final outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col1 (type: bigint) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: sum(_col1) keys: (_col0 + _col0) (type: double) mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: double) sort order: + Map-reduce partition columns: rand() (type: double) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: bigint) auto parallelism: true @@ -1822,8 +1814,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1844,8 +1836,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1864,12 +1856,12 @@ STAGE PLANS: keys: KEY._col0 (type: double) mode: partials outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: double) sort order: + Map-reduce partition columns: _col0 (type: double) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: bigint) auto parallelism: true @@ -1881,17 +1873,17 @@ STAGE PLANS: keys: KEY._col0 (type: double) mode: final outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: UDFToInteger(_col0) (type: int), UDFToInteger(_col1) (type: int) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 1 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -1905,8 +1897,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 5 + rawDataSize 15 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1938,8 +1930,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 5 + rawDataSize 15 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1959,7 +1951,6 @@ group by key + key PREHOOK: type: QUERY PREHOOK: Input: default@t1 PREHOOK: Output: default@outputtbl1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE outputTbl1 SELECT key + key, sum(cnt) from (SELECT key, count(1) as cnt FROM T1 GROUP BY key) subq1 @@ -2107,8 +2098,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 5 + rawDataSize 17 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2138,8 +2129,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2160,8 +2151,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2209,8 +2200,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 5 + rawDataSize 17 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2240,8 +2231,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2262,8 +2253,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2297,8 +2288,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 5 + rawDataSize 17 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2320,7 +2311,6 @@ SELECT key, count(1) FROM T1 GROUP BY key PREHOOK: type: QUERY PREHOOK: Input: default@t1 PREHOOK: Output: default@outputtbl1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE outputTbl1 SELECT * FROM ( SELECT key, count(1) FROM T1 GROUP BY key @@ -2484,8 +2474,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl1 numFiles 2 - numRows -1 - rawDataSize -1 + numRows 0 + rawDataSize 0 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2515,8 +2505,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2537,8 +2527,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2553,23 +2543,23 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t1 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE GatherStats: false Select Operator expressions: key (type: string) outputColumnNames: key - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) keys: (key + key) (type: double) mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: double) sort order: + Map-reduce partition columns: rand() (type: double) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: bigint) auto parallelism: true @@ -2592,8 +2582,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2614,8 +2604,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2634,12 +2624,12 @@ STAGE PLANS: keys: KEY._col0 (type: double) mode: partials outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: double) sort order: + Map-reduce partition columns: _col0 (type: double) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: bigint) auto parallelism: true @@ -2675,8 +2665,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl1 numFiles 2 - numRows -1 - rawDataSize -1 + numRows 0 + rawDataSize 0 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2710,8 +2700,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl1 numFiles 2 - numRows -1 - rawDataSize -1 + numRows 0 + rawDataSize 0 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2733,7 +2723,6 @@ SELECT key + key as key, count(1) as cnt FROM T1 GROUP BY key + key PREHOOK: type: QUERY PREHOOK: Input: default@t1 PREHOOK: Output: default@outputtbl1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE outputTbl1 SELECT * FROM ( SELECT key, count(1) as cnt FROM T1 GROUP BY key @@ -2881,31 +2870,31 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t1 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: key is not null (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string) outputColumnNames: key - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) keys: key (type: string) mode: final outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col1 (type: bigint) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) sort order: + Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE tag: 0 value expressions: _col1 (type: bigint) auto parallelism: true @@ -2928,8 +2917,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2950,8 +2939,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2966,31 +2955,31 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t1 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: key is not null (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string) outputColumnNames: key - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) keys: key (type: string) mode: final outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col1 (type: bigint) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) sort order: + Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE tag: 1 value expressions: _col1 (type: bigint) auto parallelism: true @@ -3013,8 +3002,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -3035,8 +3024,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -3057,17 +3046,17 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 {VALUE._col0} outputColumnNames: _col0, _col1, _col3 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: UDFToInteger(_col0) (type: int), UDFToInteger((_col1 + _col3)) (type: int) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 1 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -3081,8 +3070,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl1 numFiles 2 - numRows -1 - rawDataSize -1 + numRows 0 + rawDataSize 0 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -3114,8 +3103,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl1 numFiles 2 - numRows -1 - rawDataSize -1 + numRows 0 + rawDataSize 0 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -3137,7 +3126,6 @@ ON subq1.key = subq2.key PREHOOK: type: QUERY PREHOOK: Input: default@t1 PREHOOK: Output: default@outputtbl1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE outputTbl1 SELECT subq1.key, subq1.cnt+subq2.cnt FROM (SELECT key, count(1) as cnt FROM T1 GROUP BY key) subq1 @@ -3267,31 +3255,31 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t1 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: key is not null (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string) outputColumnNames: key - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) keys: key (type: string) mode: final outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col1 (type: bigint) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) sort order: + Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE tag: 0 value expressions: _col1 (type: bigint) auto parallelism: true @@ -3314,8 +3302,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -3336,8 +3324,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -3352,27 +3340,27 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t1 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: key is not null (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string), val (type: string) outputColumnNames: key, val - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) keys: key (type: string), val (type: string) mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string), _col1 (type: string) sort order: ++ Map-reduce partition columns: rand() (type: double) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col2 (type: bigint) auto parallelism: true @@ -3395,8 +3383,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -3417,8 +3405,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t1 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -3439,17 +3427,17 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 {KEY.reducesinkkey0} {VALUE._col0} {VALUE._col1} outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col1 (type: bigint), _col2 (type: string), _col3 (type: string), _col4 (type: bigint) outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -3473,12 +3461,12 @@ STAGE PLANS: keys: KEY._col0 (type: string), KEY._col1 (type: string) mode: partials outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string), _col1 (type: string) sort order: ++ Map-reduce partition columns: _col0 (type: string), _col1 (type: string) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col2 (type: bigint) auto parallelism: true @@ -3490,16 +3478,16 @@ STAGE PLANS: keys: KEY._col0 (type: string), KEY._col1 (type: string) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col1 (type: string), _col2 (type: bigint) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) sort order: + Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE tag: 1 value expressions: _col1 (type: string), _col2 (type: bigint) auto parallelism: true @@ -3525,7 +3513,6 @@ INSERT OVERWRITE TABLE T2 select key, val from T1 PREHOOK: type: QUERY PREHOOK: Input: default@t1 PREHOOK: Output: default@t2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: -- perform an insert to make sure there are 2 files INSERT OVERWRITE TABLE T2 select key, val from T1 POSTHOOK: type: QUERY @@ -3586,24 +3573,24 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t2 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE GatherStats: false Select Operator expressions: key (type: string) outputColumnNames: key - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) bucketGroup: true keys: key (type: string) mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) sort order: + Map-reduce partition columns: rand() (type: double) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: bigint) auto parallelism: true @@ -3626,8 +3613,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t2 numFiles 2 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t2 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -3648,8 +3635,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t2 numFiles 2 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t2 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -3668,12 +3655,12 @@ STAGE PLANS: keys: KEY._col0 (type: string) mode: partials outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) sort order: + Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: bigint) auto parallelism: true @@ -3685,17 +3672,17 @@ STAGE PLANS: keys: KEY._col0 (type: string) mode: final outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: UDFToInteger(_col0) (type: int), UDFToInteger(_col1) (type: int) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 1 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -3709,8 +3696,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 5 + rawDataSize 15 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -3742,8 +3729,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 5 + rawDataSize 15 serialization.ddl struct outputtbl1 { i32 key, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -3761,7 +3748,6 @@ SELECT key, count(1) FROM T2 GROUP BY key PREHOOK: type: QUERY PREHOOK: Input: default@t2 PREHOOK: Output: default@outputtbl1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE outputTbl1 SELECT key, count(1) FROM T2 GROUP BY key POSTHOOK: type: QUERY @@ -3842,28 +3828,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t2 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE GatherStats: false Select Operator expressions: key (type: string), val (type: string) outputColumnNames: key, val - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) keys: key (type: string), 1 (type: int), val (type: string) mode: final outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: UDFToInteger(_col0) (type: int), _col1 (type: int), _col2 (type: string), UDFToInteger(_col3) (type: int) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 1 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -3877,8 +3863,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl4 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 48 serialization.ddl struct outputtbl4 { i32 key1, i32 key2, string key3, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -3908,8 +3894,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t2 numFiles 2 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t2 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -3930,8 +3916,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t2 numFiles 2 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t2 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -3963,8 +3949,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl4 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 48 serialization.ddl struct outputtbl4 { i32 key1, i32 key2, string key3, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -3982,7 +3968,6 @@ SELECT key, 1, val, count(1) FROM T2 GROUP BY key, 1, val PREHOOK: type: QUERY PREHOOK: Input: default@t2 PREHOOK: Output: default@outputtbl4 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE outputTbl4 SELECT key, 1, val, count(1) FROM T2 GROUP BY key, 1, val POSTHOOK: type: QUERY @@ -4077,28 +4062,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t2 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE GatherStats: false Select Operator expressions: key (type: string), val (type: string) outputColumnNames: key, val - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) keys: key (type: string), 1 (type: int), val (type: string), 2 (type: int) mode: final outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: UDFToInteger(_col0) (type: int), _col1 (type: int), _col2 (type: string), _col3 (type: int), UDFToInteger(_col4) (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 1 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -4138,8 +4123,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t2 numFiles 2 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t2 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -4160,8 +4145,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t2 numFiles 2 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t2 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -4207,7 +4192,6 @@ SELECT key, 1, val, 2, count(1) FROM T2 GROUP BY key, 1, val, 2 PREHOOK: type: QUERY PREHOOK: Input: default@t2 PREHOOK: Output: default@outputtbl5 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE outputTbl5 SELECT key, 1, val, 2, count(1) FROM T2 GROUP BY key, 1, val, 2 POSTHOOK: type: QUERY @@ -4316,28 +4300,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t2 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE GatherStats: false Select Operator expressions: key (type: string), 1 (type: int), val (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) keys: _col0 (type: string), _col1 (type: int), _col2 (type: string) mode: final outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: UDFToInteger(_col0) (type: int), _col1 (type: int), _col2 (type: string), UDFToInteger(_col3) (type: int) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 1 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -4351,8 +4335,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl4 numFiles 2 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 48 serialization.ddl struct outputtbl4 { i32 key1, i32 key2, string key3, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -4382,8 +4366,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t2 numFiles 2 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t2 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -4404,8 +4388,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t2 numFiles 2 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t2 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -4437,8 +4421,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl4 numFiles 2 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 48 serialization.ddl struct outputtbl4 { i32 key1, i32 key2, string key3, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -4458,7 +4442,6 @@ group by key, constant, val PREHOOK: type: QUERY PREHOOK: Input: default@t2 PREHOOK: Output: default@outputtbl4 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE outputTbl4 SELECT key, constant, val, count(1) from (SELECT key, 1 as constant, val from T2)subq @@ -4594,28 +4577,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t2 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE GatherStats: false Select Operator expressions: key (type: string), 2 (type: int), val (type: string) outputColumnNames: _col0, _col3, _col2 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) keys: _col0 (type: string), _col3 (type: int), _col2 (type: string) mode: final outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: UDFToInteger(_col0) (type: int), _col1 (type: int), _col2 (type: string), UDFToInteger(_col3) (type: int) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 1 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -4629,8 +4612,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl4 numFiles 2 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 48 serialization.ddl struct outputtbl4 { i32 key1, i32 key2, string key3, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -4660,8 +4643,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t2 numFiles 2 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t2 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -4682,8 +4665,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.t2 numFiles 2 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 24 serialization.ddl struct t2 { string key, string val} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -4715,8 +4698,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.outputtbl4 numFiles 2 - numRows -1 - rawDataSize -1 + numRows 6 + rawDataSize 48 serialization.ddl struct outputtbl4 { i32 key1, i32 key2, string key3, i32 cnt} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -4739,7 +4722,6 @@ group by key, constant3, val PREHOOK: type: QUERY PREHOOK: Input: default@t2 PREHOOK: Output: default@outputtbl4 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE outputTbl4 select key, constant3, val, count(1) from ( @@ -4814,41 +4796,41 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t2 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string) outputColumnNames: key - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) bucketGroup: true keys: key (type: string) mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) sort order: + Map-reduce partition columns: rand() (type: double) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Select Operator expressions: key (type: string), val (type: string) outputColumnNames: key, val - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) keys: key (type: string), val (type: string) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: UDFToInteger(_col0) (type: int), _col1 (type: string), UDFToInteger(_col2) (type: int) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: true - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -4861,12 +4843,12 @@ STAGE PLANS: keys: KEY._col0 (type: string) mode: partials outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) sort order: + Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Reducer 3 Reduce Operator Tree: @@ -4875,14 +4857,14 @@ STAGE PLANS: keys: KEY._col0 (type: string) mode: final outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: UDFToInteger(_col0) (type: int), UDFToInteger(_col1) (type: int) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: true - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -4925,8 +4907,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@t2 PREHOOK: Output: default@dest1 PREHOOK: Output: default@dest2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM T2 INSERT OVERWRITE TABLE DEST1 SELECT key, count(1) GROUP BY key INSERT OVERWRITE TABLE DEST2 SELECT key, val, count(1) GROUP BY key, val @@ -4998,48 +4978,48 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t2 - Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (key = 8) (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: val (type: string) outputColumnNames: _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: '8' (type: string) outputColumnNames: _col0 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) bucketGroup: true keys: _col0 (type: string) mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) sort order: + Map-reduce partition columns: rand() (type: double) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Select Operator expressions: '8' (type: string), _col1 (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) keys: _col0 (type: string), _col1 (type: string) mode: final outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: UDFToInteger(_col0) (type: int), _col1 (type: string), UDFToInteger(_col2) (type: int) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: true - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -5052,12 +5032,12 @@ STAGE PLANS: keys: KEY._col0 (type: string) mode: partials outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) sort order: + Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Reducer 3 Reduce Operator Tree: @@ -5066,14 +5046,14 @@ STAGE PLANS: keys: KEY._col0 (type: string) mode: final outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: UDFToInteger(_col0) (type: int), UDFToInteger(_col1) (type: int) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: true - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -5116,8 +5096,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@t2 PREHOOK: Output: default@dest1 PREHOOK: Output: default@dest2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM (select key, val from T2 where key = 8) x INSERT OVERWRITE TABLE DEST1 SELECT key, count(1) GROUP BY key INSERT OVERWRITE TABLE DEST2 SELECT key, val, count(1) GROUP BY key, val diff --git ql/src/test/results/clientpositive/spark/innerjoin.q.out ql/src/test/results/clientpositive/spark/innerjoin.q.out index 5e1a2aa..acfe946 100644 --- ql/src/test/results/clientpositive/spark/innerjoin.q.out +++ ql/src/test/results/clientpositive/spark/innerjoin.q.out @@ -102,7 +102,6 @@ INSERT OVERWRITE TABLE dest_j1 SELECT src1.key, src2.value PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest_j1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM src src1 INNER JOIN src src2 ON (src1.key = src2.key) INSERT OVERWRITE TABLE dest_j1 SELECT src1.key, src2.value POSTHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/spark/input12.q.out ql/src/test/results/clientpositive/spark/input12.q.out index 8ef35c4..4ef6ed4 100644 --- ql/src/test/results/clientpositive/spark/input12.q.out +++ ql/src/test/results/clientpositive/spark/input12.q.out @@ -158,9 +158,6 @@ PREHOOK: Input: default@src PREHOOK: Output: default@dest1 PREHOOK: Output: default@dest2 PREHOOK: Output: default@dest3@ds=2008-04-08/hr=12 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM src INSERT OVERWRITE TABLE dest1 SELECT src.* WHERE src.key < 100 INSERT OVERWRITE TABLE dest2 SELECT src.key, src.value WHERE src.key >= 100 and src.key < 200 diff --git ql/src/test/results/clientpositive/spark/input14.q.out ql/src/test/results/clientpositive/spark/input14.q.out index e7d4db6..676caec 100644 --- ql/src/test/results/clientpositive/spark/input14.q.out +++ ql/src/test/results/clientpositive/spark/input14.q.out @@ -103,7 +103,6 @@ INSERT OVERWRITE TABLE dest1 SELECT tmap.tkey, tmap.tvalue WHERE tmap.tkey < 100 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM ( FROM src SELECT TRANSFORM(src.key, src.value) diff --git ql/src/test/results/clientpositive/spark/input17.q.out ql/src/test/results/clientpositive/spark/input17.q.out index 0882a29..8922f15 100644 --- ql/src/test/results/clientpositive/spark/input17.q.out +++ ql/src/test/results/clientpositive/spark/input17.q.out @@ -100,7 +100,6 @@ INSERT OVERWRITE TABLE dest1 SELECT tmap.tkey, tmap.tvalue PREHOOK: type: QUERY PREHOOK: Input: default@src_thrift PREHOOK: Output: default@dest1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM ( FROM src_thrift SELECT TRANSFORM(src_thrift.aint + src_thrift.lint[0], src_thrift.lintstring[0]) diff --git ql/src/test/results/clientpositive/spark/input18.q.out ql/src/test/results/clientpositive/spark/input18.q.out index 802fb0a..46df37a 100644 --- ql/src/test/results/clientpositive/spark/input18.q.out +++ ql/src/test/results/clientpositive/spark/input18.q.out @@ -103,7 +103,6 @@ INSERT OVERWRITE TABLE dest1 SELECT tmap.key, regexp_replace(tmap.value,'\t','+' PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM ( FROM src SELECT TRANSFORM(src.key, src.value, 1+2, 3+4) diff --git ql/src/test/results/clientpositive/spark/input1_limit.q.out ql/src/test/results/clientpositive/spark/input1_limit.q.out index 33ecd07..e9f8d5e 100644 --- ql/src/test/results/clientpositive/spark/input1_limit.q.out +++ ql/src/test/results/clientpositive/spark/input1_limit.q.out @@ -156,8 +156,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 PREHOOK: Output: default@dest2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM src INSERT OVERWRITE TABLE dest1 SELECT src.key, src.value WHERE src.key < 100 LIMIT 10 INSERT OVERWRITE TABLE dest2 SELECT src.key, src.value WHERE src.key < 100 LIMIT 5 diff --git ql/src/test/results/clientpositive/spark/input_part2.q.out ql/src/test/results/clientpositive/spark/input_part2.q.out index b9e85fe..587b4ac 100644 --- ql/src/test/results/clientpositive/spark/input_part2.q.out +++ ql/src/test/results/clientpositive/spark/input_part2.q.out @@ -377,8 +377,6 @@ PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 PREHOOK: Output: default@dest1 PREHOOK: Output: default@dest2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM srcpart INSERT OVERWRITE TABLE dest1 SELECT srcpart.key, srcpart.value, srcpart.hr, srcpart.ds WHERE srcpart.key < 100 and srcpart.ds = '2008-04-08' and srcpart.hr = '12' INSERT OVERWRITE TABLE dest2 SELECT srcpart.key, srcpart.value, srcpart.hr, srcpart.ds WHERE srcpart.key < 100 and srcpart.ds = '2008-04-09' and srcpart.hr = '12' diff --git ql/src/test/results/clientpositive/spark/insert1.q.out ql/src/test/results/clientpositive/spark/insert1.q.out index 49fb1d4..74b1bc3 100644 --- ql/src/test/results/clientpositive/spark/insert1.q.out +++ ql/src/test/results/clientpositive/spark/insert1.q.out @@ -22,7 +22,6 @@ PREHOOK: query: insert overwrite table insert1 select a.key, a.value from insert PREHOOK: type: QUERY PREHOOK: Input: default@insert2 PREHOOK: Output: default@insert1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table insert1 select a.key, a.value from insert2 a WHERE (a.key=-1) POSTHOOK: type: QUERY POSTHOOK: Input: default@insert2 @@ -367,7 +366,6 @@ PREHOOK: query: INSERT OVERWRITE TABLE result SELECT 'db2_insert1' FROM default. PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: db2@result -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE result SELECT 'db2_insert1' FROM default.src LIMIT 1 POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -377,7 +375,6 @@ PREHOOK: query: INSERT INTO TABLE result SELECT 'db2_insert2' FROM default.src L PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: db2@result -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT INTO TABLE result SELECT 'db2_insert2' FROM default.src LIMIT 1 POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -417,7 +414,6 @@ PREHOOK: query: INSERT OVERWRITE TABLE db1.result SELECT 'db1_insert1' FROM src PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: db1@result -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE db1.result SELECT 'db1_insert1' FROM src LIMIT 1 POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -427,7 +423,6 @@ PREHOOK: query: INSERT INTO TABLE db1.result SELECT 'db1_insert2' FROM src LIMIT PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: db1@result -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT INTO TABLE db1.result SELECT 'db1_insert2' FROM src LIMIT 1 POSTHOOK: type: QUERY POSTHOOK: Input: default@src diff --git ql/src/test/results/clientpositive/spark/insert_into1.q.out ql/src/test/results/clientpositive/spark/insert_into1.q.out index e9be658..58d6d69 100644 --- ql/src/test/results/clientpositive/spark/insert_into1.q.out +++ ql/src/test/results/clientpositive/spark/insert_into1.q.out @@ -85,7 +85,6 @@ PREHOOK: query: INSERT INTO TABLE insert_into1 SELECT * from src LIMIT 100 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@insert_into1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT INTO TABLE insert_into1 SELECT * from src LIMIT 100 POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -112,54 +111,12 @@ POSTHOOK: query: explain select count(*) from insert_into1 POSTHOOK: type: QUERY STAGE DEPENDENCIES: - Stage-1 is a root stage - Stage-0 depends on stages: Stage-1 + Stage-0 is a root stage STAGE PLANS: - Stage: Stage-1 - Spark - Edges: - Reducer 2 <- Map 1 (GROUP, 1) -#### A masked pattern was here #### - Vertices: - Map 1 - Map Operator Tree: - TableScan - alias: insert_into1 - Statistics: Num rows: 0 Data size: 1176 Basic stats: PARTIAL Column stats: COMPLETE - Select Operator - Statistics: Num rows: 0 Data size: 1176 Basic stats: PARTIAL Column stats: COMPLETE - Group By Operator - aggregations: count() - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint) - Reducer 2 - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0) - mode: mergepartial - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: bigint) - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Stage: Stage-0 Fetch Operator - limit: -1 + limit: 1 Processor Tree: ListSink @@ -247,7 +204,6 @@ PREHOOK: query: INSERT INTO TABLE insert_into1 SELECT * FROM src LIMIT 100 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@insert_into1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT INTO TABLE insert_into1 SELECT * FROM src LIMIT 100 POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -274,54 +230,12 @@ POSTHOOK: query: explain SELECT COUNT(*) FROM insert_into1 POSTHOOK: type: QUERY STAGE DEPENDENCIES: - Stage-1 is a root stage - Stage-0 depends on stages: Stage-1 + Stage-0 is a root stage STAGE PLANS: - Stage: Stage-1 - Spark - Edges: - Reducer 2 <- Map 1 (GROUP, 1) -#### A masked pattern was here #### - Vertices: - Map 1 - Map Operator Tree: - TableScan - alias: insert_into1 - Statistics: Num rows: 0 Data size: 2352 Basic stats: PARTIAL Column stats: COMPLETE - Select Operator - Statistics: Num rows: 0 Data size: 2352 Basic stats: PARTIAL Column stats: COMPLETE - Group By Operator - aggregations: count() - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint) - Reducer 2 - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0) - mode: mergepartial - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: bigint) - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Stage: Stage-0 Fetch Operator - limit: -1 + limit: 1 Processor Tree: ListSink @@ -409,7 +323,6 @@ PREHOOK: query: INSERT OVERWRITE TABLE insert_into1 SELECT * FROM src LIMIT 10 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@insert_into1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE insert_into1 SELECT * FROM src LIMIT 10 POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -436,54 +349,12 @@ POSTHOOK: query: explain SELECT COUNT(*) FROM insert_into1 POSTHOOK: type: QUERY STAGE DEPENDENCIES: - Stage-1 is a root stage - Stage-0 depends on stages: Stage-1 + Stage-0 is a root stage STAGE PLANS: - Stage: Stage-1 - Spark - Edges: - Reducer 2 <- Map 1 (GROUP, 1) -#### A masked pattern was here #### - Vertices: - Map 1 - Map Operator Tree: - TableScan - alias: insert_into1 - Statistics: Num rows: -1 Data size: 114 Basic stats: PARTIAL Column stats: COMPLETE - Select Operator - Statistics: Num rows: -1 Data size: 114 Basic stats: PARTIAL Column stats: COMPLETE - Group By Operator - aggregations: count() - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint) - Reducer 2 - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0) - mode: mergepartial - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: bigint) - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Stage: Stage-0 Fetch Operator - limit: -1 + limit: 1 Processor Tree: ListSink diff --git ql/src/test/results/clientpositive/spark/insert_into2.q.out ql/src/test/results/clientpositive/spark/insert_into2.q.out index 5c8e9c7..640f531 100644 --- ql/src/test/results/clientpositive/spark/insert_into2.q.out +++ ql/src/test/results/clientpositive/spark/insert_into2.q.out @@ -91,7 +91,6 @@ PREHOOK: query: INSERT INTO TABLE insert_into2 PARTITION (ds='1') SELECT * FROM PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@insert_into2@ds=1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT INTO TABLE insert_into2 PARTITION (ds='1') SELECT * FROM src limit 100 POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -105,73 +104,28 @@ POSTHOOK: query: explain select count (*) from insert_into2 where ds = '1' POSTHOOK: type: QUERY STAGE DEPENDENCIES: - Stage-1 is a root stage - Stage-0 depends on stages: Stage-1 + Stage-0 is a root stage STAGE PLANS: - Stage: Stage-1 - Spark - Edges: - Reducer 2 <- Map 1 (GROUP, 1) -#### A masked pattern was here #### - Vertices: - Map 1 - Map Operator Tree: - TableScan - alias: insert_into2 - Statistics: Num rows: 0 Data size: 1176 Basic stats: PARTIAL Column stats: NONE - Select Operator - Statistics: Num rows: 0 Data size: 1176 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 - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: bigint) - Reducer 2 - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0) - mode: mergepartial - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col0 (type: bigint) - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Stage: Stage-0 Fetch Operator - limit: -1 + limit: 1 Processor Tree: ListSink PREHOOK: query: select count (*) from insert_into2 where ds = '1' PREHOOK: type: QUERY PREHOOK: Input: default@insert_into2 -PREHOOK: Input: default@insert_into2@ds=1 #### A masked pattern was here #### POSTHOOK: query: select count (*) from insert_into2 where ds = '1' POSTHOOK: type: QUERY POSTHOOK: Input: default@insert_into2 -POSTHOOK: Input: default@insert_into2@ds=1 #### A masked pattern was here #### 100 PREHOOK: query: INSERT INTO TABLE insert_into2 PARTITION (ds='1') SELECT * FROM src limit 100 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@insert_into2@ds=1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT INTO TABLE insert_into2 PARTITION (ds='1') SELECT * FROM src limit 100 POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -185,66 +139,22 @@ POSTHOOK: query: explain SELECT COUNT(*) FROM insert_into2 WHERE ds='1' POSTHOOK: type: QUERY STAGE DEPENDENCIES: - Stage-1 is a root stage - Stage-0 depends on stages: Stage-1 + Stage-0 is a root stage STAGE PLANS: - Stage: Stage-1 - Spark - Edges: - Reducer 2 <- Map 1 (GROUP, 1) -#### A masked pattern was here #### - Vertices: - Map 1 - Map Operator Tree: - TableScan - alias: insert_into2 - Statistics: Num rows: 0 Data size: 2352 Basic stats: PARTIAL Column stats: NONE - Select Operator - Statistics: Num rows: 0 Data size: 2352 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 - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: bigint) - Reducer 2 - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0) - mode: mergepartial - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col0 (type: bigint) - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Stage: Stage-0 Fetch Operator - limit: -1 + limit: 1 Processor Tree: ListSink PREHOOK: query: SELECT COUNT(*) FROM insert_into2 WHERE ds='1' PREHOOK: type: QUERY PREHOOK: Input: default@insert_into2 -PREHOOK: Input: default@insert_into2@ds=1 #### A masked pattern was here #### POSTHOOK: query: SELECT COUNT(*) FROM insert_into2 WHERE ds='1' POSTHOOK: type: QUERY POSTHOOK: Input: default@insert_into2 -POSTHOOK: Input: default@insert_into2@ds=1 #### A masked pattern was here #### 200 PREHOOK: query: SELECT SUM(HASH(c)) FROM ( @@ -342,7 +252,6 @@ PREHOOK: query: INSERT OVERWRITE TABLE insert_into2 PARTITION (ds='2') PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@insert_into2@ds=2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE insert_into2 PARTITION (ds='2') SELECT * FROM src LIMIT 100 POSTHOOK: type: QUERY @@ -374,66 +283,22 @@ POSTHOOK: query: explain SELECT COUNT(*) FROM insert_into2 WHERE ds='2' POSTHOOK: type: QUERY STAGE DEPENDENCIES: - Stage-1 is a root stage - Stage-0 depends on stages: Stage-1 + Stage-0 is a root stage STAGE PLANS: - Stage: Stage-1 - Spark - Edges: - Reducer 2 <- Map 1 (GROUP, 1) -#### A masked pattern was here #### - Vertices: - Map 1 - Map Operator Tree: - TableScan - alias: insert_into2 - Statistics: Num rows: 0 Data size: 1176 Basic stats: PARTIAL Column stats: NONE - Select Operator - Statistics: Num rows: 0 Data size: 1176 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 - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: bigint) - Reducer 2 - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0) - mode: mergepartial - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col0 (type: bigint) - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Stage: Stage-0 Fetch Operator - limit: -1 + limit: 1 Processor Tree: ListSink PREHOOK: query: SELECT COUNT(*) FROM insert_into2 WHERE ds='2' PREHOOK: type: QUERY PREHOOK: Input: default@insert_into2 -PREHOOK: Input: default@insert_into2@ds=2 #### A masked pattern was here #### POSTHOOK: query: SELECT COUNT(*) FROM insert_into2 WHERE ds='2' POSTHOOK: type: QUERY POSTHOOK: Input: default@insert_into2 -POSTHOOK: Input: default@insert_into2@ds=2 #### A masked pattern was here #### 100 PREHOOK: query: EXPLAIN INSERT OVERWRITE TABLE insert_into2 PARTITION (ds='2') @@ -516,7 +381,6 @@ PREHOOK: query: INSERT OVERWRITE TABLE insert_into2 PARTITION (ds='2') PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@insert_into2@ds=2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE insert_into2 PARTITION (ds='2') SELECT * FROM src LIMIT 50 POSTHOOK: type: QUERY @@ -548,66 +412,22 @@ POSTHOOK: query: explain SELECT COUNT(*) FROM insert_into2 WHERE ds='2' POSTHOOK: type: QUERY STAGE DEPENDENCIES: - Stage-1 is a root stage - Stage-0 depends on stages: Stage-1 + Stage-0 is a root stage STAGE PLANS: - Stage: Stage-1 - Spark - Edges: - Reducer 2 <- Map 1 (GROUP, 1) -#### A masked pattern was here #### - Vertices: - Map 1 - Map Operator Tree: - TableScan - alias: insert_into2 - Statistics: Num rows: 0 Data size: 586 Basic stats: PARTIAL Column stats: NONE - Select Operator - Statistics: Num rows: 0 Data size: 586 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 - Reduce Output Operator - sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: bigint) - Reducer 2 - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0) - mode: mergepartial - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col0 (type: bigint) - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Stage: Stage-0 Fetch Operator - limit: -1 + limit: 1 Processor Tree: ListSink PREHOOK: query: SELECT COUNT(*) FROM insert_into2 WHERE ds='2' PREHOOK: type: QUERY PREHOOK: Input: default@insert_into2 -PREHOOK: Input: default@insert_into2@ds=2 #### A masked pattern was here #### POSTHOOK: query: SELECT COUNT(*) FROM insert_into2 WHERE ds='2' POSTHOOK: type: QUERY POSTHOOK: Input: default@insert_into2 -POSTHOOK: Input: default@insert_into2@ds=2 #### A masked pattern was here #### 50 PREHOOK: query: DROP TABLE insert_into2 diff --git ql/src/test/results/clientpositive/spark/insert_into3.q.out ql/src/test/results/clientpositive/spark/insert_into3.q.out index 6c0111d..04c9858 100644 --- ql/src/test/results/clientpositive/spark/insert_into3.q.out +++ ql/src/test/results/clientpositive/spark/insert_into3.q.out @@ -143,8 +143,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@insert_into3a PREHOOK: Output: default@insert_into3b -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM src INSERT INTO TABLE insert_into3a SELECT * ORDER BY key, value LIMIT 50 INSERT INTO TABLE insert_into3b SELECT * ORDER BY key, value LIMIT 100 POSTHOOK: type: QUERY @@ -308,8 +306,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@insert_into3a PREHOOK: Output: default@insert_into3b -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM src INSERT OVERWRITE TABLE insert_into3a SELECT * LIMIT 10 INSERT INTO TABLE insert_into3b SELECT * LIMIT 10 POSTHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/spark/join1.q.out ql/src/test/results/clientpositive/spark/join1.q.out index da63bee..ec726c3 100644 --- ql/src/test/results/clientpositive/spark/join1.q.out +++ ql/src/test/results/clientpositive/spark/join1.q.out @@ -98,7 +98,6 @@ INSERT OVERWRITE TABLE dest_j1 SELECT src1.key, src2.value PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest_j1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM src src1 JOIN src src2 ON (src1.key = src2.key) INSERT OVERWRITE TABLE dest_j1 SELECT src1.key, src2.value POSTHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/spark/join14.q.out ql/src/test/results/clientpositive/spark/join14.q.out index f44c5dd..cf3e7e4 100644 --- ql/src/test/results/clientpositive/spark/join14.q.out +++ ql/src/test/results/clientpositive/spark/join14.q.out @@ -105,7 +105,6 @@ PREHOOK: Input: default@srcpart PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 PREHOOK: Output: default@dest1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM src JOIN srcpart ON src.key = srcpart.key AND srcpart.ds = '2008-04-08' and src.key > 100 INSERT OVERWRITE TABLE dest1 SELECT src.key, srcpart.value POSTHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/spark/join17.q.out ql/src/test/results/clientpositive/spark/join17.q.out index 6d68ed3..aab7bf2 100644 --- ql/src/test/results/clientpositive/spark/join17.q.out +++ ql/src/test/results/clientpositive/spark/join17.q.out @@ -275,7 +275,6 @@ INSERT OVERWRITE TABLE dest1 SELECT src1.*, src2.* PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM src src1 JOIN src src2 ON (src1.key = src2.key) INSERT OVERWRITE TABLE dest1 SELECT src1.*, src2.* POSTHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/spark/join2.q.out ql/src/test/results/clientpositive/spark/join2.q.out index 56e0fe9..22f23c7 100644 --- ql/src/test/results/clientpositive/spark/join2.q.out +++ ql/src/test/results/clientpositive/spark/join2.q.out @@ -131,7 +131,6 @@ INSERT OVERWRITE TABLE dest_j2 SELECT src1.key, src3.value PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest_j2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM src src1 JOIN src src2 ON (src1.key = src2.key) JOIN src src3 ON (src1.key + src2.key = src3.key) INSERT OVERWRITE TABLE dest_j2 SELECT src1.key, src3.value POSTHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/spark/join24.q.out ql/src/test/results/clientpositive/spark/join24.q.out index 0de60d0..ea7fdfe 100644 --- ql/src/test/results/clientpositive/spark/join24.q.out +++ ql/src/test/results/clientpositive/spark/join24.q.out @@ -11,7 +11,6 @@ SELECT a.key, count(1) FROM src a group by a.key PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@tst1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE tst1 SELECT a.key, count(1) FROM src a group by a.key POSTHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/spark/join25.q.out ql/src/test/results/clientpositive/spark/join25.q.out index f2c8457..60a0675 100644 --- ql/src/test/results/clientpositive/spark/join25.q.out +++ ql/src/test/results/clientpositive/spark/join25.q.out @@ -107,7 +107,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Input: default@src1 PREHOOK: Output: default@dest_j1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE dest_j1 SELECT /*+ MAPJOIN(x) */ x.key, x.value, y.value FROM src1 x JOIN src y ON (x.key = y.key) diff --git ql/src/test/results/clientpositive/spark/join26.q.out ql/src/test/results/clientpositive/spark/join26.q.out index 3b010a0..217074f 100644 --- ql/src/test/results/clientpositive/spark/join26.q.out +++ ql/src/test/results/clientpositive/spark/join26.q.out @@ -399,7 +399,6 @@ PREHOOK: Input: default@src1 PREHOOK: Input: default@srcpart PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 PREHOOK: Output: default@dest_j1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE dest_j1 SELECT /*+ MAPJOIN(x,y) */ x.key, z.value, y.value FROM src1 x JOIN src y ON (x.key = y.key) diff --git ql/src/test/results/clientpositive/spark/join27.q.out ql/src/test/results/clientpositive/spark/join27.q.out index 2e1f61a..98bff59 100644 --- ql/src/test/results/clientpositive/spark/join27.q.out +++ ql/src/test/results/clientpositive/spark/join27.q.out @@ -106,7 +106,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Input: default@src1 PREHOOK: Output: default@dest_j1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE dest_j1 SELECT /*+ MAPJOIN(x) */ x.key, x.value, y.value FROM src1 x JOIN src y ON (x.value = y.value) diff --git ql/src/test/results/clientpositive/spark/join3.q.out ql/src/test/results/clientpositive/spark/join3.q.out index e61f724..662989c 100644 --- ql/src/test/results/clientpositive/spark/join3.q.out +++ ql/src/test/results/clientpositive/spark/join3.q.out @@ -113,7 +113,6 @@ INSERT OVERWRITE TABLE dest1 SELECT src1.key, src3.value PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM src src1 JOIN src src2 ON (src1.key = src2.key) JOIN src src3 ON (src1.key = src3.key) INSERT OVERWRITE TABLE dest1 SELECT src1.key, src3.value POSTHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/spark/join4.q.out ql/src/test/results/clientpositive/spark/join4.q.out index 1d48393..e56ebcf 100644 --- ql/src/test/results/clientpositive/spark/join4.q.out +++ ql/src/test/results/clientpositive/spark/join4.q.out @@ -140,7 +140,6 @@ INSERT OVERWRITE TABLE dest1 SELECT c.c1, c.c2, c.c3, c.c4 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM ( FROM ( diff --git ql/src/test/results/clientpositive/spark/join5.q.out ql/src/test/results/clientpositive/spark/join5.q.out index 247750e..eb60466 100644 --- ql/src/test/results/clientpositive/spark/join5.q.out +++ ql/src/test/results/clientpositive/spark/join5.q.out @@ -140,7 +140,6 @@ INSERT OVERWRITE TABLE dest1 SELECT c.c1, c.c2, c.c3, c.c4 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM ( FROM ( diff --git ql/src/test/results/clientpositive/spark/join6.q.out ql/src/test/results/clientpositive/spark/join6.q.out index 966fcf2..64c2221 100644 --- ql/src/test/results/clientpositive/spark/join6.q.out +++ ql/src/test/results/clientpositive/spark/join6.q.out @@ -140,7 +140,6 @@ INSERT OVERWRITE TABLE dest1 SELECT c.c1, c.c2, c.c3, c.c4 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM ( FROM ( diff --git ql/src/test/results/clientpositive/spark/join7.q.out ql/src/test/results/clientpositive/spark/join7.q.out index d7e8bd1..cde86b2 100644 --- ql/src/test/results/clientpositive/spark/join7.q.out +++ ql/src/test/results/clientpositive/spark/join7.q.out @@ -175,7 +175,6 @@ INSERT OVERWRITE TABLE dest1 SELECT c.c1, c.c2, c.c3, c.c4, c.c5, c.c6 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM ( FROM ( diff --git ql/src/test/results/clientpositive/spark/join8.q.out ql/src/test/results/clientpositive/spark/join8.q.out index 31115c3..e413e77 100644 --- ql/src/test/results/clientpositive/spark/join8.q.out +++ ql/src/test/results/clientpositive/spark/join8.q.out @@ -143,7 +143,6 @@ INSERT OVERWRITE TABLE dest1 SELECT c.c1, c.c2, c.c3, c.c4 where c.c3 IS NULL AN PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM ( FROM ( diff --git ql/src/test/results/clientpositive/spark/join9.q.out ql/src/test/results/clientpositive/spark/join9.q.out index 5553ad2..2ea2034 100644 --- ql/src/test/results/clientpositive/spark/join9.q.out +++ ql/src/test/results/clientpositive/spark/join9.q.out @@ -294,7 +294,6 @@ PREHOOK: Input: default@src PREHOOK: Input: default@srcpart PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 PREHOOK: Output: default@dest1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM srcpart src1 JOIN src src2 ON (src1.key = src2.key) INSERT OVERWRITE TABLE dest1 SELECT src1.key, src2.value where src1.ds = '2008-04-08' and src1.hr = '12' POSTHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/spark/join_nullsafe.q.out ql/src/test/results/clientpositive/spark/join_nullsafe.q.out index c8d4413..57d24b3 100644 --- ql/src/test/results/clientpositive/spark/join_nullsafe.q.out +++ ql/src/test/results/clientpositive/spark/join_nullsafe.q.out @@ -653,8 +653,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@smb_input PREHOOK: Output: default@smb_input1 PREHOOK: Output: default@smb_input2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: from smb_input insert overwrite table smb_input1 select * insert overwrite table smb_input2 select * diff --git ql/src/test/results/clientpositive/spark/load_dyn_part1.q.out ql/src/test/results/clientpositive/spark/load_dyn_part1.q.out index 5d2622d..eced900 100644 --- ql/src/test/results/clientpositive/spark/load_dyn_part1.q.out +++ ql/src/test/results/clientpositive/spark/load_dyn_part1.q.out @@ -151,8 +151,6 @@ PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 PREHOOK: Output: default@nzhang_part1 PREHOOK: Output: default@nzhang_part2@ds=2008-12-31 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: from srcpart insert overwrite table nzhang_part1 partition (ds, hr) select key, value, ds, hr where ds <= '2008-04-08' insert overwrite table nzhang_part2 partition(ds='2008-12-31', hr) select key, value, hr where ds > '2008-04-08' diff --git ql/src/test/results/clientpositive/spark/load_dyn_part10.q.out ql/src/test/results/clientpositive/spark/load_dyn_part10.q.out index 9b70e4c..9ce5b8b 100644 --- ql/src/test/results/clientpositive/spark/load_dyn_part10.q.out +++ ql/src/test/results/clientpositive/spark/load_dyn_part10.q.out @@ -101,7 +101,6 @@ PREHOOK: Input: default@srcpart PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 PREHOOK: Output: default@nzhang_part10@ds=2008-12-31 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: from srcpart insert overwrite table nzhang_part10 partition(ds='2008-12-31', hr) select key, value, hr where ds > '2008-04-08' POSTHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/spark/load_dyn_part11.q.out ql/src/test/results/clientpositive/spark/load_dyn_part11.q.out index aa77469..225a6c9 100644 --- ql/src/test/results/clientpositive/spark/load_dyn_part11.q.out +++ ql/src/test/results/clientpositive/spark/load_dyn_part11.q.out @@ -42,7 +42,6 @@ PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 PREHOOK: Output: default@nzhang_part@ds=2010-03-03 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table nzhang_part partition (ds="2010-03-03", hr) select key, value, hr from srcpart where ds is not null and hr is not null POSTHOOK: type: QUERY POSTHOOK: Input: default@srcpart diff --git ql/src/test/results/clientpositive/spark/load_dyn_part12.q.out ql/src/test/results/clientpositive/spark/load_dyn_part12.q.out index 0be0c69..bb51159 100644 --- ql/src/test/results/clientpositive/spark/load_dyn_part12.q.out +++ ql/src/test/results/clientpositive/spark/load_dyn_part12.q.out @@ -42,7 +42,6 @@ PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 PREHOOK: Output: default@nzhang_part12@ds=2010-03-03 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table nzhang_part12 partition (ds="2010-03-03", hr) select key, value, cast(hr*2 as int) from srcpart where ds is not null and hr is not null POSTHOOK: type: QUERY POSTHOOK: Input: default@srcpart diff --git ql/src/test/results/clientpositive/spark/load_dyn_part13.q.out ql/src/test/results/clientpositive/spark/load_dyn_part13.q.out index eb3e01a..acf45ee 100644 --- ql/src/test/results/clientpositive/spark/load_dyn_part13.q.out +++ ql/src/test/results/clientpositive/spark/load_dyn_part13.q.out @@ -141,7 +141,6 @@ select * from ( PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@nzhang_part13@ds=2010-03-03 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table nzhang_part13 partition (ds="2010-03-03", hr) select * from ( select key, value, '22' diff --git ql/src/test/results/clientpositive/spark/load_dyn_part14.q.out ql/src/test/results/clientpositive/spark/load_dyn_part14.q.out index 5dd5fad..65f98a9 100644 --- ql/src/test/results/clientpositive/spark/load_dyn_part14.q.out +++ ql/src/test/results/clientpositive/spark/load_dyn_part14.q.out @@ -200,7 +200,6 @@ select key, value from ( PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@nzhang_part14 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table nzhang_part14 partition(value) select key, value from ( select 'k1' as key, cast(null as string) as value from src limit 2 diff --git ql/src/test/results/clientpositive/spark/load_dyn_part15.q.out ql/src/test/results/clientpositive/spark/load_dyn_part15.q.out index d85fd69..b105497 100644 --- ql/src/test/results/clientpositive/spark/load_dyn_part15.q.out +++ ql/src/test/results/clientpositive/spark/load_dyn_part15.q.out @@ -19,7 +19,6 @@ SELECT key, part_key FROM src LATERAL VIEW explode(array("1","{2","3]")) myTable PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@load_dyn_part15_test -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE load_dyn_part15_test PARTITION(part_key) SELECT key, part_key FROM src LATERAL VIEW explode(array("1","{2","3]")) myTable AS part_key POSTHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/spark/load_dyn_part2.q.out ql/src/test/results/clientpositive/spark/load_dyn_part2.q.out index f8f8971..e52588c 100644 --- ql/src/test/results/clientpositive/spark/load_dyn_part2.q.out +++ ql/src/test/results/clientpositive/spark/load_dyn_part2.q.out @@ -105,7 +105,6 @@ PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 PREHOOK: Output: default@nzhang_part_bucket@ds=2010-03-23 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table nzhang_part_bucket partition (ds='2010-03-23', hr) select key, value, hr from srcpart where ds is not null and hr is not null POSTHOOK: type: QUERY POSTHOOK: Input: default@srcpart diff --git ql/src/test/results/clientpositive/spark/load_dyn_part3.q.out ql/src/test/results/clientpositive/spark/load_dyn_part3.q.out index b6ae54c..5cc23a7 100644 --- ql/src/test/results/clientpositive/spark/load_dyn_part3.q.out +++ ql/src/test/results/clientpositive/spark/load_dyn_part3.q.out @@ -100,7 +100,6 @@ PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 PREHOOK: Output: default@nzhang_part3 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table nzhang_part3 partition (ds, hr) select key, value, ds, hr from srcpart where ds is not null and hr is not null POSTHOOK: type: QUERY POSTHOOK: Input: default@srcpart diff --git ql/src/test/results/clientpositive/spark/load_dyn_part4.q.out ql/src/test/results/clientpositive/spark/load_dyn_part4.q.out index d8c1412..7439dbc 100644 --- ql/src/test/results/clientpositive/spark/load_dyn_part4.q.out +++ ql/src/test/results/clientpositive/spark/load_dyn_part4.q.out @@ -42,7 +42,6 @@ PREHOOK: query: insert overwrite table nzhang_part4 partition (ds='2008-04-08', PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@nzhang_part4@ds=2008-04-08/hr=existing_value -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table nzhang_part4 partition (ds='2008-04-08', hr='existing_value') select key, value from src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -111,7 +110,6 @@ PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 PREHOOK: Output: default@nzhang_part4 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table nzhang_part4 partition (ds, hr) select key, value, ds, hr from srcpart where ds is not null and hr is not null POSTHOOK: type: QUERY POSTHOOK: Input: default@srcpart diff --git ql/src/test/results/clientpositive/spark/load_dyn_part5.q.out ql/src/test/results/clientpositive/spark/load_dyn_part5.q.out index b5c83cb..8d38814 100644 --- ql/src/test/results/clientpositive/spark/load_dyn_part5.q.out +++ ql/src/test/results/clientpositive/spark/load_dyn_part5.q.out @@ -78,7 +78,6 @@ PREHOOK: query: insert overwrite table nzhang_part5 partition (value) select key PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@nzhang_part5 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table nzhang_part5 partition (value) select key, value from src POSTHOOK: type: QUERY POSTHOOK: Input: default@src diff --git ql/src/test/results/clientpositive/spark/load_dyn_part6.q.out ql/src/test/results/clientpositive/spark/load_dyn_part6.q.out index f23dbe5..fce8eff 100644 --- ql/src/test/results/clientpositive/spark/load_dyn_part6.q.out +++ ql/src/test/results/clientpositive/spark/load_dyn_part6.q.out @@ -42,7 +42,6 @@ PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 PREHOOK: Output: default@nzhang_part6@ds=2010-03-03 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table nzhang_part6 partition (ds="2010-03-03", hr) select key, value, hr from srcpart where ds is not null and hr is not null POSTHOOK: type: QUERY POSTHOOK: Input: default@srcpart diff --git ql/src/test/results/clientpositive/spark/load_dyn_part7.q.out ql/src/test/results/clientpositive/spark/load_dyn_part7.q.out index 6a12103..02f9327 100644 --- ql/src/test/results/clientpositive/spark/load_dyn_part7.q.out +++ ql/src/test/results/clientpositive/spark/load_dyn_part7.q.out @@ -39,7 +39,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@srcpart PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 PREHOOK: Output: default@nzhang_part7@ds=2010-03-03/hr=12 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table nzhang_part7 partition (ds='2010-03-03', hr='12') select key, value from srcpart where ds = '2008-04-08' and hr = '12' POSTHOOK: type: QUERY POSTHOOK: Input: default@srcpart diff --git ql/src/test/results/clientpositive/spark/load_dyn_part8.q.out ql/src/test/results/clientpositive/spark/load_dyn_part8.q.out index a12c75c..bc42891 100644 --- ql/src/test/results/clientpositive/spark/load_dyn_part8.q.out +++ ql/src/test/results/clientpositive/spark/load_dyn_part8.q.out @@ -472,8 +472,6 @@ PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 PREHOOK: Output: default@nzhang_part8 PREHOOK: Output: default@nzhang_part8@ds=2008-12-31 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: from srcpart insert overwrite table nzhang_part8 partition (ds, hr) select key, value, ds, hr where ds <= '2008-04-08' insert overwrite table nzhang_part8 partition(ds='2008-12-31', hr) select key, value, hr where ds > '2008-04-08' diff --git ql/src/test/results/clientpositive/spark/load_dyn_part9.q.out ql/src/test/results/clientpositive/spark/load_dyn_part9.q.out index d158180..52af0c9 100644 --- ql/src/test/results/clientpositive/spark/load_dyn_part9.q.out +++ ql/src/test/results/clientpositive/spark/load_dyn_part9.q.out @@ -101,7 +101,6 @@ PREHOOK: Input: default@srcpart PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 PREHOOK: Output: default@nzhang_part9 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: from srcpart insert overwrite table nzhang_part9 partition (ds, hr) select key, value, ds, hr where ds <= '2008-04-08' POSTHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/spark/mapreduce1.q.out ql/src/test/results/clientpositive/spark/mapreduce1.q.out index 1824126..d328632 100644 --- ql/src/test/results/clientpositive/spark/mapreduce1.q.out +++ ql/src/test/results/clientpositive/spark/mapreduce1.q.out @@ -97,7 +97,6 @@ SORT BY ten, one PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM src INSERT OVERWRITE TABLE dest1 MAP src.key, CAST(src.key / 10 AS INT), CAST(src.key % 10 AS INT), src.value diff --git ql/src/test/results/clientpositive/spark/mapreduce2.q.out ql/src/test/results/clientpositive/spark/mapreduce2.q.out index 792a0c8..eeaaf14 100644 --- ql/src/test/results/clientpositive/spark/mapreduce2.q.out +++ ql/src/test/results/clientpositive/spark/mapreduce2.q.out @@ -93,7 +93,6 @@ DISTRIBUTE BY tvalue, tkey PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM src INSERT OVERWRITE TABLE dest1 MAP src.key, CAST(src.key / 10 AS INT), CAST(src.key % 10 AS INT), src.value diff --git ql/src/test/results/clientpositive/spark/merge1.q.out ql/src/test/results/clientpositive/spark/merge1.q.out index c50a80b..f93fe77 100644 --- ql/src/test/results/clientpositive/spark/merge1.q.out +++ ql/src/test/results/clientpositive/spark/merge1.q.out @@ -144,7 +144,6 @@ select key, count(1) from src group by key PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table dest1 select key, count(1) from src group by key POSTHOOK: type: QUERY @@ -497,7 +496,6 @@ PREHOOK: query: insert overwrite table test_src partition(ds='101') select * fro PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@test_src@ds=101 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table test_src partition(ds='101') select * from src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -508,7 +506,6 @@ PREHOOK: query: insert overwrite table test_src partition(ds='102') select * fro PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@test_src@ds=102 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table test_src partition(ds='102') select * from src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -541,14 +538,14 @@ STAGE PLANS: Map Operator Tree: TableScan alias: test_src - Statistics: Num rows: 116 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string) outputColumnNames: _col0 - Statistics: Num rows: 116 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 116 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -622,7 +619,6 @@ PREHOOK: Input: default@test_src PREHOOK: Input: default@test_src@ds=101 PREHOOK: Input: default@test_src@ds=102 PREHOOK: Output: default@dest1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table dest1 select key from test_src POSTHOOK: type: QUERY POSTHOOK: Input: default@test_src @@ -656,14 +652,14 @@ STAGE PLANS: Map Operator Tree: TableScan alias: test_src - Statistics: Num rows: 116 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string) outputColumnNames: _col0 - Statistics: Num rows: 116 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 116 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -737,7 +733,6 @@ PREHOOK: Input: default@test_src PREHOOK: Input: default@test_src@ds=101 PREHOOK: Input: default@test_src@ds=102 PREHOOK: Output: default@dest1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table dest1 select key from test_src POSTHOOK: type: QUERY POSTHOOK: Input: default@test_src diff --git ql/src/test/results/clientpositive/spark/merge2.q.out ql/src/test/results/clientpositive/spark/merge2.q.out index aec97a3..24f5b2f 100644 --- ql/src/test/results/clientpositive/spark/merge2.q.out +++ ql/src/test/results/clientpositive/spark/merge2.q.out @@ -144,7 +144,6 @@ select key, count(1) from src group by key PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@test1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table test1 select key, count(1) from src group by key POSTHOOK: type: QUERY @@ -497,7 +496,6 @@ PREHOOK: query: insert overwrite table test_src partition(ds='101') select * fro PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@test_src@ds=101 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table test_src partition(ds='101') select * from src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -508,7 +506,6 @@ PREHOOK: query: insert overwrite table test_src partition(ds='102') select * fro PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@test_src@ds=102 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table test_src partition(ds='102') select * from src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -541,14 +538,14 @@ STAGE PLANS: Map Operator Tree: TableScan alias: test_src - Statistics: Num rows: 116 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string) outputColumnNames: _col0 - Statistics: Num rows: 116 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 116 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -622,7 +619,6 @@ PREHOOK: Input: default@test_src PREHOOK: Input: default@test_src@ds=101 PREHOOK: Input: default@test_src@ds=102 PREHOOK: Output: default@test1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table test1 select key from test_src POSTHOOK: type: QUERY POSTHOOK: Input: default@test_src @@ -656,14 +652,14 @@ STAGE PLANS: Map Operator Tree: TableScan alias: test_src - Statistics: Num rows: 116 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string) outputColumnNames: _col0 - Statistics: Num rows: 116 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 116 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -737,7 +733,6 @@ PREHOOK: Input: default@test_src PREHOOK: Input: default@test_src@ds=101 PREHOOK: Input: default@test_src@ds=102 PREHOOK: Output: default@test1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table test1 select key from test_src POSTHOOK: type: QUERY POSTHOOK: Input: default@test_src diff --git ql/src/test/results/clientpositive/spark/multi_insert.q.out ql/src/test/results/clientpositive/spark/multi_insert.q.out index 2b9f90e..4b7c43b 100644 --- ql/src/test/results/clientpositive/spark/multi_insert.q.out +++ ql/src/test/results/clientpositive/spark/multi_insert.q.out @@ -113,8 +113,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@src_multi1 PREHOOK: Output: default@src_multi2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: from src insert overwrite table src_multi1 select * where key < 10 insert overwrite table src_multi2 select * where key > 10 and key < 20 @@ -256,8 +254,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@src_multi1 PREHOOK: Output: default@src_multi2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: from src insert overwrite table src_multi1 select * where key < 10 insert overwrite table src_multi2 select * where key > 10 and key < 20 @@ -399,8 +395,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@src_multi1 PREHOOK: Output: default@src_multi2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: from src insert overwrite table src_multi1 select * where key < 10 insert overwrite table src_multi2 select * where key > 10 and key < 20 @@ -542,8 +536,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@src_multi1 PREHOOK: Output: default@src_multi2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: from src insert overwrite table src_multi1 select * where key < 10 insert overwrite table src_multi2 select * where key > 10 and key < 20 @@ -713,8 +705,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@src_multi1 PREHOOK: Output: default@src_multi2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: from src insert overwrite table src_multi1 select * where key < 10 group by key, value insert overwrite table src_multi2 select * where key > 10 and key < 20 group by key, value @@ -877,8 +867,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@src_multi1 PREHOOK: Output: default@src_multi2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: from src insert overwrite table src_multi1 select * where key < 10 group by key, value insert overwrite table src_multi2 select * where key > 10 and key < 20 group by key, value @@ -1041,8 +1029,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@src_multi1 PREHOOK: Output: default@src_multi2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: from src insert overwrite table src_multi1 select * where key < 10 group by key, value insert overwrite table src_multi2 select * where key > 10 and key < 20 group by key, value @@ -1205,8 +1191,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@src_multi1 PREHOOK: Output: default@src_multi2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: from src insert overwrite table src_multi1 select * where key < 10 group by key, value insert overwrite table src_multi2 select * where key > 10 and key < 20 group by key, value @@ -1372,8 +1356,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@src_multi1 PREHOOK: Output: default@src_multi2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: from (select * from src union all select * from src) s insert overwrite table src_multi1 select * where key < 10 insert overwrite table src_multi2 select * where key > 10 and key < 20 @@ -1565,8 +1547,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@src_multi1 PREHOOK: Output: default@src_multi2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: from (select * from src union all select * from src) s insert overwrite table src_multi1 select * where key < 10 insert overwrite table src_multi2 select * where key > 10 and key < 20 @@ -1758,8 +1738,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@src_multi1 PREHOOK: Output: default@src_multi2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: from (select * from src union all select * from src) s insert overwrite table src_multi1 select * where key < 10 insert overwrite table src_multi2 select * where key > 10 and key < 20 @@ -1951,8 +1929,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@src_multi1 PREHOOK: Output: default@src_multi2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: from (select * from src union all select * from src) s insert overwrite table src_multi1 select * where key < 10 insert overwrite table src_multi2 select * where key > 10 and key < 20 diff --git ql/src/test/results/clientpositive/spark/multi_insert_gby.q.out ql/src/test/results/clientpositive/spark/multi_insert_gby.q.out index 7d6d58b..6e5c3a9 100644 --- ql/src/test/results/clientpositive/spark/multi_insert_gby.q.out +++ ql/src/test/results/clientpositive/spark/multi_insert_gby.q.out @@ -149,8 +149,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@e1 PREHOOK: Output: default@e2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM src INSERT OVERWRITE TABLE e1 SELECT key, COUNT(*) WHERE key>450 GROUP BY key @@ -340,8 +338,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@e1 PREHOOK: Output: default@e2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM src INSERT OVERWRITE TABLE e1 SELECT key, COUNT(*) WHERE key>450 GROUP BY key diff --git ql/src/test/results/clientpositive/spark/multi_insert_gby2.q.out ql/src/test/results/clientpositive/spark/multi_insert_gby2.q.out index fca3e1d..5b98d7f 100644 --- ql/src/test/results/clientpositive/spark/multi_insert_gby2.q.out +++ ql/src/test/results/clientpositive/spark/multi_insert_gby2.q.out @@ -147,8 +147,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@e1 PREHOOK: Output: default@e2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM (select key, cast(key as double) as value from src order by key) a INSERT OVERWRITE TABLE e1 SELECT COUNT(*) diff --git ql/src/test/results/clientpositive/spark/multi_insert_gby3.q.out ql/src/test/results/clientpositive/spark/multi_insert_gby3.q.out index ce78fba..4afb763 100644 --- ql/src/test/results/clientpositive/spark/multi_insert_gby3.q.out +++ ql/src/test/results/clientpositive/spark/multi_insert_gby3.q.out @@ -283,8 +283,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@e1 PREHOOK: Output: default@e2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM (select key, cast(key as double) as keyD, value from src order by key) a INSERT OVERWRITE TABLE e1 SELECT key, COUNT(distinct value) group by key @@ -942,8 +940,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@e1 PREHOOK: Output: default@e2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM (select key, cast(key as double) as keyD, value from src order by key) a INSERT OVERWRITE TABLE e2 SELECT key, sum(keyD), value group by key, value diff --git ql/src/test/results/clientpositive/spark/multi_insert_mixed.q.out ql/src/test/results/clientpositive/spark/multi_insert_mixed.q.out index 819b265..c2bc349 100644 --- ql/src/test/results/clientpositive/spark/multi_insert_mixed.q.out +++ ql/src/test/results/clientpositive/spark/multi_insert_mixed.q.out @@ -226,9 +226,6 @@ PREHOOK: Input: default@src PREHOOK: Output: default@src_multi1 PREHOOK: Output: default@src_multi2 PREHOOK: Output: default@src_multi3 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: from src insert overwrite table src_multi1 select key, count(1) group by key order by key insert overwrite table src_multi2 select value, count(1) group by value order by value diff --git ql/src/test/results/clientpositive/spark/optimize_nullscan.q.out ql/src/test/results/clientpositive/spark/optimize_nullscan.q.out index d9de8d9..cda8f53 100644 --- ql/src/test/results/clientpositive/spark/optimize_nullscan.q.out +++ ql/src/test/results/clientpositive/spark/optimize_nullscan.q.out @@ -247,9 +247,9 @@ STAGE PLANS: tag: 0 auto parallelism: true Path -> Alias: - -mr-10002default.src{} [src] + -mr-10003default.src{} [src] Path -> Partition: - -mr-10002default.src{} + -mr-10003default.src{} Partition base file name: src input format: org.apache.hadoop.hive.ql.io.OneNullRowInputFormat @@ -294,7 +294,7 @@ STAGE PLANS: name: default.src name: default.src Truncated Path -> Alias: - -mr-10002default.src{} [src] + -mr-10003default.src{} [src] Map 3 Map Operator Tree: TableScan @@ -315,12 +315,12 @@ STAGE PLANS: value expressions: _col0 (type: string) auto parallelism: false Path -> Alias: - -mr-10003default.srcpart{ds=2008-04-08, hr=11} [srcpart] - -mr-10004default.srcpart{ds=2008-04-08, hr=12} [srcpart] - -mr-10005default.srcpart{ds=2008-04-09, hr=11} [srcpart] - -mr-10006default.srcpart{ds=2008-04-09, hr=12} [srcpart] + -mr-10004default.srcpart{ds=2008-04-08, hr=11} [srcpart] + -mr-10005default.srcpart{ds=2008-04-08, hr=12} [srcpart] + -mr-10006default.srcpart{ds=2008-04-09, hr=11} [srcpart] + -mr-10007default.srcpart{ds=2008-04-09, hr=12} [srcpart] Path -> Partition: - -mr-10003default.srcpart{ds=2008-04-08, hr=11} + -mr-10004default.srcpart{ds=2008-04-08, hr=11} Partition base file name: hr=11 input format: org.apache.hadoop.hive.ql.io.OneNullRowInputFormat @@ -366,7 +366,7 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.srcpart name: default.srcpart - -mr-10004default.srcpart{ds=2008-04-08, hr=12} + -mr-10005default.srcpart{ds=2008-04-08, hr=12} Partition base file name: hr=12 input format: org.apache.hadoop.hive.ql.io.OneNullRowInputFormat @@ -412,7 +412,7 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.srcpart name: default.srcpart - -mr-10005default.srcpart{ds=2008-04-09, hr=11} + -mr-10006default.srcpart{ds=2008-04-09, hr=11} Partition base file name: hr=11 input format: org.apache.hadoop.hive.ql.io.OneNullRowInputFormat @@ -458,7 +458,7 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.srcpart name: default.srcpart - -mr-10006default.srcpart{ds=2008-04-09, hr=12} + -mr-10007default.srcpart{ds=2008-04-09, hr=12} Partition base file name: hr=12 input format: org.apache.hadoop.hive.ql.io.OneNullRowInputFormat @@ -505,10 +505,10 @@ STAGE PLANS: name: default.srcpart name: default.srcpart Truncated Path -> Alias: - -mr-10003default.srcpart{ds=2008-04-08, hr=11} [srcpart] - -mr-10004default.srcpart{ds=2008-04-08, hr=12} [srcpart] - -mr-10005default.srcpart{ds=2008-04-09, hr=11} [srcpart] - -mr-10006default.srcpart{ds=2008-04-09, hr=12} [srcpart] + -mr-10004default.srcpart{ds=2008-04-08, hr=11} [srcpart] + -mr-10005default.srcpart{ds=2008-04-08, hr=12} [srcpart] + -mr-10006default.srcpart{ds=2008-04-09, hr=11} [srcpart] + -mr-10007default.srcpart{ds=2008-04-09, hr=12} [srcpart] Reducer 2 Needs Tagging: true Reduce Operator Tree: @@ -893,9 +893,9 @@ STAGE PLANS: value expressions: _col0 (type: bigint) auto parallelism: false Path -> Alias: - -mr-10002default.src{} [src] + -mr-10003default.src{} [src] Path -> Partition: - -mr-10002default.src{} + -mr-10003default.src{} Partition base file name: src input format: org.apache.hadoop.hive.ql.io.OneNullRowInputFormat @@ -940,7 +940,7 @@ STAGE PLANS: name: default.src name: default.src Truncated Path -> Alias: - -mr-10002default.src{} [src] + -mr-10003default.src{} [src] Reducer 2 Needs Tagging: false Reduce Operator Tree: @@ -1034,8 +1034,8 @@ POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 #### A masked pattern was here #### -2000 0 +2000 PREHOOK: query: explain extended select * from (select key from src where false) a left outer join (select value from srcpart limit 0) b PREHOOK: type: QUERY @@ -1123,9 +1123,9 @@ STAGE PLANS: value expressions: _col0 (type: string) auto parallelism: false Path -> Alias: - -mr-10002default.src{} [src] + -mr-10003default.src{} [src] Path -> Partition: - -mr-10002default.src{} + -mr-10003default.src{} Partition base file name: src input format: org.apache.hadoop.hive.ql.io.OneNullRowInputFormat @@ -1170,7 +1170,7 @@ STAGE PLANS: name: default.src name: default.src Truncated Path -> Alias: - -mr-10002default.src{} [src] + -mr-10003default.src{} [src] Map 3 Map Operator Tree: TableScan @@ -1191,12 +1191,12 @@ STAGE PLANS: value expressions: _col0 (type: string) auto parallelism: false Path -> Alias: - -mr-10003default.srcpart{ds=2008-04-08, hr=11} [srcpart] - -mr-10004default.srcpart{ds=2008-04-08, hr=12} [srcpart] - -mr-10005default.srcpart{ds=2008-04-09, hr=11} [srcpart] - -mr-10006default.srcpart{ds=2008-04-09, hr=12} [srcpart] + -mr-10004default.srcpart{ds=2008-04-08, hr=11} [srcpart] + -mr-10005default.srcpart{ds=2008-04-08, hr=12} [srcpart] + -mr-10006default.srcpart{ds=2008-04-09, hr=11} [srcpart] + -mr-10007default.srcpart{ds=2008-04-09, hr=12} [srcpart] Path -> Partition: - -mr-10003default.srcpart{ds=2008-04-08, hr=11} + -mr-10004default.srcpart{ds=2008-04-08, hr=11} Partition base file name: hr=11 input format: org.apache.hadoop.hive.ql.io.OneNullRowInputFormat @@ -1242,7 +1242,7 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.srcpart name: default.srcpart - -mr-10004default.srcpart{ds=2008-04-08, hr=12} + -mr-10005default.srcpart{ds=2008-04-08, hr=12} Partition base file name: hr=12 input format: org.apache.hadoop.hive.ql.io.OneNullRowInputFormat @@ -1288,7 +1288,7 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.srcpart name: default.srcpart - -mr-10005default.srcpart{ds=2008-04-09, hr=11} + -mr-10006default.srcpart{ds=2008-04-09, hr=11} Partition base file name: hr=11 input format: org.apache.hadoop.hive.ql.io.OneNullRowInputFormat @@ -1334,7 +1334,7 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.srcpart name: default.srcpart - -mr-10006default.srcpart{ds=2008-04-09, hr=12} + -mr-10007default.srcpart{ds=2008-04-09, hr=12} Partition base file name: hr=12 input format: org.apache.hadoop.hive.ql.io.OneNullRowInputFormat @@ -1381,10 +1381,10 @@ STAGE PLANS: name: default.srcpart name: default.srcpart Truncated Path -> Alias: - -mr-10003default.srcpart{ds=2008-04-08, hr=11} [srcpart] - -mr-10004default.srcpart{ds=2008-04-08, hr=12} [srcpart] - -mr-10005default.srcpart{ds=2008-04-09, hr=11} [srcpart] - -mr-10006default.srcpart{ds=2008-04-09, hr=12} [srcpart] + -mr-10004default.srcpart{ds=2008-04-08, hr=11} [srcpart] + -mr-10005default.srcpart{ds=2008-04-08, hr=12} [srcpart] + -mr-10006default.srcpart{ds=2008-04-09, hr=11} [srcpart] + -mr-10007default.srcpart{ds=2008-04-09, hr=12} [srcpart] Reducer 2 Needs Tagging: true Reduce Operator Tree: diff --git ql/src/test/results/clientpositive/spark/parallel.q.out ql/src/test/results/clientpositive/spark/parallel.q.out index 32d7ff1..640f557 100644 --- ql/src/test/results/clientpositive/spark/parallel.q.out +++ ql/src/test/results/clientpositive/spark/parallel.q.out @@ -154,8 +154,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@src_a PREHOOK: Output: default@src_b -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: from (select key, value from src group by key, value) s insert overwrite table src_a select s.key, s.value group by s.key, s.value insert overwrite table src_b select s.key, s.value group by s.key, s.value @@ -808,8 +806,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@src_a PREHOOK: Output: default@src_b -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: from (select key, value from src group by key, value) s insert overwrite table src_a select s.key, s.value group by s.key, s.value insert overwrite table src_b select s.key, s.value group by s.key, s.value diff --git ql/src/test/results/clientpositive/spark/parallel_join1.q.out ql/src/test/results/clientpositive/spark/parallel_join1.q.out index 8ba3abb..04730bd 100644 --- ql/src/test/results/clientpositive/spark/parallel_join1.q.out +++ ql/src/test/results/clientpositive/spark/parallel_join1.q.out @@ -102,7 +102,6 @@ INSERT OVERWRITE TABLE dest_j1 SELECT src1.key, src2.value PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest_j1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM src src1 JOIN src src2 ON (src1.key = src2.key) INSERT OVERWRITE TABLE dest_j1 SELECT src1.key, src2.value POSTHOOK: type: QUERY @@ -118,226 +117,6 @@ POSTHOOK: query: SELECT dest_j1.* FROM dest_j1 POSTHOOK: type: QUERY POSTHOOK: Input: default@dest_j1 #### A masked pattern was here #### -0 val_0 -0 val_0 -0 val_0 -0 val_0 -0 val_0 -0 val_0 -0 val_0 -0 val_0 -0 val_0 -103 val_103 -103 val_103 -103 val_103 -103 val_103 -11 val_11 -114 val_114 -118 val_118 -118 val_118 -118 val_118 -118 val_118 -125 val_125 -125 val_125 -125 val_125 -125 val_125 -129 val_129 -129 val_129 -129 val_129 -129 val_129 -136 val_136 -143 val_143 -15 val_15 -15 val_15 -15 val_15 -15 val_15 -150 val_150 -158 val_158 -165 val_165 -165 val_165 -165 val_165 -165 val_165 -169 val_169 -169 val_169 -169 val_169 -169 val_169 -169 val_169 -169 val_169 -169 val_169 -169 val_169 -169 val_169 -169 val_169 -169 val_169 -169 val_169 -169 val_169 -169 val_169 -169 val_169 -169 val_169 -172 val_172 -172 val_172 -172 val_172 -172 val_172 -176 val_176 -176 val_176 -176 val_176 -176 val_176 -183 val_183 -187 val_187 -187 val_187 -187 val_187 -187 val_187 -187 val_187 -187 val_187 -187 val_187 -187 val_187 -187 val_187 -19 val_19 -190 val_190 -194 val_194 -202 val_202 -213 val_213 -213 val_213 -213 val_213 -213 val_213 -217 val_217 -217 val_217 -217 val_217 -217 val_217 -224 val_224 -224 val_224 -224 val_224 -224 val_224 -228 val_228 -235 val_235 -239 val_239 -239 val_239 -239 val_239 -239 val_239 -242 val_242 -242 val_242 -242 val_242 -242 val_242 -257 val_257 -26 val_26 -26 val_26 -26 val_26 -26 val_26 -260 val_260 -275 val_275 -282 val_282 -282 val_282 -282 val_282 -282 val_282 -286 val_286 -305 val_305 -309 val_309 -309 val_309 -309 val_309 -309 val_309 -316 val_316 -316 val_316 -316 val_316 -316 val_316 -316 val_316 -316 val_316 -316 val_316 -316 val_316 -316 val_316 -323 val_323 -327 val_327 -327 val_327 -327 val_327 -327 val_327 -327 val_327 -327 val_327 -327 val_327 -327 val_327 -327 val_327 -33 val_33 -338 val_338 -341 val_341 -345 val_345 -356 val_356 -367 val_367 -367 val_367 -367 val_367 -367 val_367 -37 val_37 -37 val_37 -37 val_37 -37 val_37 -374 val_374 -378 val_378 -389 val_389 -392 val_392 -396 val_396 -396 val_396 -396 val_396 -396 val_396 -396 val_396 -396 val_396 -396 val_396 -396 val_396 -396 val_396 -4 val_4 -400 val_400 -404 val_404 -404 val_404 -404 val_404 -404 val_404 -411 val_411 -419 val_419 -437 val_437 -44 val_44 -444 val_444 -448 val_448 -455 val_455 -459 val_459 -459 val_459 -459 val_459 -459 val_459 -462 val_462 -462 val_462 -462 val_462 -462 val_462 -466 val_466 -466 val_466 -466 val_466 -466 val_466 -466 val_466 -466 val_466 -466 val_466 -466 val_466 -466 val_466 -477 val_477 -480 val_480 -480 val_480 -480 val_480 -480 val_480 -480 val_480 -480 val_480 -480 val_480 -480 val_480 -480 val_480 -484 val_484 -491 val_491 -495 val_495 -51 val_51 -51 val_51 -51 val_51 -51 val_51 -66 val_66 -77 val_77 -8 val_8 -80 val_80 -84 val_84 -84 val_84 -84 val_84 -84 val_84 -95 val_95 -95 val_95 -95 val_95 -95 val_95 10 val_10 113 val_113 113 val_113 @@ -642,266 +421,6 @@ POSTHOOK: Input: default@dest_j1 98 val_98 98 val_98 98 val_98 -105 val_105 -116 val_116 -134 val_134 -134 val_134 -134 val_134 -134 val_134 -138 val_138 -138 val_138 -138 val_138 -138 val_138 -138 val_138 -138 val_138 -138 val_138 -138 val_138 -138 val_138 -138 val_138 -138 val_138 -138 val_138 -138 val_138 -138 val_138 -138 val_138 -138 val_138 -145 val_145 -149 val_149 -149 val_149 -149 val_149 -149 val_149 -152 val_152 -152 val_152 -152 val_152 -152 val_152 -156 val_156 -163 val_163 -167 val_167 -167 val_167 -167 val_167 -167 val_167 -167 val_167 -167 val_167 -167 val_167 -167 val_167 -167 val_167 -17 val_17 -170 val_170 -174 val_174 -174 val_174 -174 val_174 -174 val_174 -178 val_178 -181 val_181 -189 val_189 -192 val_192 -196 val_196 -2 val_2 -20 val_20 -200 val_200 -200 val_200 -200 val_200 -200 val_200 -208 val_208 -208 val_208 -208 val_208 -208 val_208 -208 val_208 -208 val_208 -208 val_208 -208 val_208 -208 val_208 -219 val_219 -219 val_219 -219 val_219 -219 val_219 -222 val_222 -226 val_226 -233 val_233 -233 val_233 -233 val_233 -233 val_233 -237 val_237 -237 val_237 -237 val_237 -237 val_237 -24 val_24 -24 val_24 -24 val_24 -24 val_24 -244 val_244 -248 val_248 -255 val_255 -255 val_255 -255 val_255 -255 val_255 -262 val_262 -266 val_266 -273 val_273 -273 val_273 -273 val_273 -273 val_273 -273 val_273 -273 val_273 -273 val_273 -273 val_273 -273 val_273 -277 val_277 -277 val_277 -277 val_277 -277 val_277 -277 val_277 -277 val_277 -277 val_277 -277 val_277 -277 val_277 -277 val_277 -277 val_277 -277 val_277 -277 val_277 -277 val_277 -277 val_277 -277 val_277 -28 val_28 -280 val_280 -280 val_280 -280 val_280 -280 val_280 -284 val_284 -288 val_288 -288 val_288 -288 val_288 -288 val_288 -291 val_291 -307 val_307 -307 val_307 -307 val_307 -307 val_307 -310 val_310 -318 val_318 -318 val_318 -318 val_318 -318 val_318 -318 val_318 -318 val_318 -318 val_318 -318 val_318 -318 val_318 -321 val_321 -321 val_321 -321 val_321 -321 val_321 -325 val_325 -325 val_325 -325 val_325 -325 val_325 -332 val_332 -336 val_336 -35 val_35 -35 val_35 -35 val_35 -35 val_35 -35 val_35 -35 val_35 -35 val_35 -35 val_35 -35 val_35 -365 val_365 -369 val_369 -369 val_369 -369 val_369 -369 val_369 -369 val_369 -369 val_369 -369 val_369 -369 val_369 -369 val_369 -394 val_394 -402 val_402 -406 val_406 -406 val_406 -406 val_406 -406 val_406 -406 val_406 -406 val_406 -406 val_406 -406 val_406 -406 val_406 -406 val_406 -406 val_406 -406 val_406 -406 val_406 -406 val_406 -406 val_406 -406 val_406 -413 val_413 -413 val_413 -413 val_413 -413 val_413 -417 val_417 -417 val_417 -417 val_417 -417 val_417 -417 val_417 -417 val_417 -417 val_417 -417 val_417 -417 val_417 -42 val_42 -42 val_42 -42 val_42 -42 val_42 -424 val_424 -424 val_424 -424 val_424 -424 val_424 -431 val_431 -431 val_431 -431 val_431 -431 val_431 -431 val_431 -431 val_431 -431 val_431 -431 val_431 -431 val_431 -435 val_435 -439 val_439 -439 val_439 -439 val_439 -439 val_439 -446 val_446 -453 val_453 -457 val_457 -460 val_460 -468 val_468 -468 val_468 -468 val_468 -468 val_468 -468 val_468 -468 val_468 -468 val_468 -468 val_468 -468 val_468 -468 val_468 -468 val_468 -468 val_468 -468 val_468 -468 val_468 -468 val_468 -468 val_468 -475 val_475 -479 val_479 -482 val_482 -493 val_493 -497 val_497 -53 val_53 -57 val_57 -64 val_64 -82 val_82 -86 val_86 -97 val_97 -97 val_97 -97 val_97 -97 val_97 100 val_100 100 val_100 100 val_100 @@ -1146,3 +665,483 @@ POSTHOOK: Input: default@dest_j1 9 val_9 92 val_92 96 val_96 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +103 val_103 +103 val_103 +103 val_103 +103 val_103 +11 val_11 +114 val_114 +118 val_118 +118 val_118 +118 val_118 +118 val_118 +125 val_125 +125 val_125 +125 val_125 +125 val_125 +129 val_129 +129 val_129 +129 val_129 +129 val_129 +136 val_136 +143 val_143 +15 val_15 +15 val_15 +15 val_15 +15 val_15 +150 val_150 +158 val_158 +165 val_165 +165 val_165 +165 val_165 +165 val_165 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +172 val_172 +172 val_172 +172 val_172 +172 val_172 +176 val_176 +176 val_176 +176 val_176 +176 val_176 +183 val_183 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +19 val_19 +190 val_190 +194 val_194 +202 val_202 +213 val_213 +213 val_213 +213 val_213 +213 val_213 +217 val_217 +217 val_217 +217 val_217 +217 val_217 +224 val_224 +224 val_224 +224 val_224 +224 val_224 +228 val_228 +235 val_235 +239 val_239 +239 val_239 +239 val_239 +239 val_239 +242 val_242 +242 val_242 +242 val_242 +242 val_242 +257 val_257 +26 val_26 +26 val_26 +26 val_26 +26 val_26 +260 val_260 +275 val_275 +282 val_282 +282 val_282 +282 val_282 +282 val_282 +286 val_286 +305 val_305 +309 val_309 +309 val_309 +309 val_309 +309 val_309 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +323 val_323 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +33 val_33 +338 val_338 +341 val_341 +345 val_345 +356 val_356 +367 val_367 +367 val_367 +367 val_367 +367 val_367 +37 val_37 +37 val_37 +37 val_37 +37 val_37 +374 val_374 +378 val_378 +389 val_389 +392 val_392 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +4 val_4 +400 val_400 +404 val_404 +404 val_404 +404 val_404 +404 val_404 +411 val_411 +419 val_419 +437 val_437 +44 val_44 +444 val_444 +448 val_448 +455 val_455 +459 val_459 +459 val_459 +459 val_459 +459 val_459 +462 val_462 +462 val_462 +462 val_462 +462 val_462 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +477 val_477 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +484 val_484 +491 val_491 +495 val_495 +51 val_51 +51 val_51 +51 val_51 +51 val_51 +66 val_66 +77 val_77 +8 val_8 +80 val_80 +84 val_84 +84 val_84 +84 val_84 +84 val_84 +95 val_95 +95 val_95 +95 val_95 +95 val_95 +105 val_105 +116 val_116 +134 val_134 +134 val_134 +134 val_134 +134 val_134 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +145 val_145 +149 val_149 +149 val_149 +149 val_149 +149 val_149 +152 val_152 +152 val_152 +152 val_152 +152 val_152 +156 val_156 +163 val_163 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +17 val_17 +170 val_170 +174 val_174 +174 val_174 +174 val_174 +174 val_174 +178 val_178 +181 val_181 +189 val_189 +192 val_192 +196 val_196 +2 val_2 +20 val_20 +200 val_200 +200 val_200 +200 val_200 +200 val_200 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +219 val_219 +219 val_219 +219 val_219 +219 val_219 +222 val_222 +226 val_226 +233 val_233 +233 val_233 +233 val_233 +233 val_233 +237 val_237 +237 val_237 +237 val_237 +237 val_237 +24 val_24 +24 val_24 +24 val_24 +24 val_24 +244 val_244 +248 val_248 +255 val_255 +255 val_255 +255 val_255 +255 val_255 +262 val_262 +266 val_266 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +28 val_28 +280 val_280 +280 val_280 +280 val_280 +280 val_280 +284 val_284 +288 val_288 +288 val_288 +288 val_288 +288 val_288 +291 val_291 +307 val_307 +307 val_307 +307 val_307 +307 val_307 +310 val_310 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +321 val_321 +321 val_321 +321 val_321 +321 val_321 +325 val_325 +325 val_325 +325 val_325 +325 val_325 +332 val_332 +336 val_336 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +365 val_365 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +394 val_394 +402 val_402 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +413 val_413 +413 val_413 +413 val_413 +413 val_413 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +42 val_42 +42 val_42 +42 val_42 +42 val_42 +424 val_424 +424 val_424 +424 val_424 +424 val_424 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +435 val_435 +439 val_439 +439 val_439 +439 val_439 +439 val_439 +446 val_446 +453 val_453 +457 val_457 +460 val_460 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +475 val_475 +479 val_479 +482 val_482 +493 val_493 +497 val_497 +53 val_53 +57 val_57 +64 val_64 +82 val_82 +86 val_86 +97 val_97 +97 val_97 +97 val_97 +97 val_97 diff --git ql/src/test/results/clientpositive/spark/pcr.q.out ql/src/test/results/clientpositive/spark/pcr.q.out index 4e9244f..16aac1f 100644 --- ql/src/test/results/clientpositive/spark/pcr.q.out +++ ql/src/test/results/clientpositive/spark/pcr.q.out @@ -22,7 +22,6 @@ PREHOOK: query: insert overwrite table pcr_t1 partition (ds='2000-04-08') select PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@pcr_t1@ds=2000-04-08 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table pcr_t1 partition (ds='2000-04-08') select * from src where key < 20 order by key POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -33,7 +32,6 @@ PREHOOK: query: insert overwrite table pcr_t1 partition (ds='2000-04-09') select PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@pcr_t1@ds=2000-04-09 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table pcr_t1 partition (ds='2000-04-09') select * from src where key < 20 order by key POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -44,7 +42,6 @@ PREHOOK: query: insert overwrite table pcr_t1 partition (ds='2000-04-10') select PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@pcr_t1@ds=2000-04-10 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table pcr_t1 partition (ds='2000-04-10') select * from src where key < 20 order by key POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -110,20 +107,20 @@ STAGE PLANS: Map Operator Tree: TableScan alias: pcr_t1 - Statistics: Num rows: 2 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: (key < 5) (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 13 Data size: 104 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int), value (type: string), ds (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 13 Data size: 104 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int), _col2 (type: string) sort order: ++ - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 13 Data size: 104 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: string) auto parallelism: false @@ -146,10 +143,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t1 numFiles 1 - numRows -1 + numRows 20 partition_columns ds partition_columns.types string - rawDataSize -1 + rawDataSize 160 serialization.ddl struct pcr_t1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -191,10 +188,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t1 numFiles 1 - numRows -1 + numRows 20 partition_columns ds partition_columns.types string - rawDataSize -1 + rawDataSize 160 serialization.ddl struct pcr_t1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -229,13 +226,13 @@ STAGE PLANS: Select Operator expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string), KEY.reducesinkkey1 (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 13 Data size: 104 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 13 Data size: 104 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -333,20 +330,20 @@ STAGE PLANS: Map Operator Tree: TableScan alias: pcr_t1 - Statistics: Num rows: 3 Data size: 540 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 60 Data size: 480 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: ((ds <= '2000-04-09') or (key < 5)) (type: boolean) - Statistics: Num rows: 2 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int), value (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 2 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) sort order: + - Statistics: Num rows: 2 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: string) auto parallelism: false @@ -369,10 +366,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t1 numFiles 1 - numRows -1 + numRows 20 partition_columns ds partition_columns.types string - rawDataSize -1 + rawDataSize 160 serialization.ddl struct pcr_t1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -414,10 +411,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t1 numFiles 1 - numRows -1 + numRows 20 partition_columns ds partition_columns.types string - rawDataSize -1 + rawDataSize 160 serialization.ddl struct pcr_t1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -459,10 +456,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t1 numFiles 1 - numRows -1 + numRows 20 partition_columns ds partition_columns.types string - rawDataSize -1 + rawDataSize 160 serialization.ddl struct pcr_t1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -498,13 +495,13 @@ STAGE PLANS: Select Operator expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 2 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 2 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -650,20 +647,20 @@ STAGE PLANS: Map Operator Tree: TableScan alias: pcr_t1 - Statistics: Num rows: 2 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: ((key < 5) and (value <> 'val_2')) (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 13 Data size: 104 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int), value (type: string), ds (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 13 Data size: 104 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int), _col2 (type: string) sort order: ++ - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 13 Data size: 104 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: string) auto parallelism: false @@ -686,10 +683,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t1 numFiles 1 - numRows -1 + numRows 20 partition_columns ds partition_columns.types string - rawDataSize -1 + rawDataSize 160 serialization.ddl struct pcr_t1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -731,10 +728,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t1 numFiles 1 - numRows -1 + numRows 20 partition_columns ds partition_columns.types string - rawDataSize -1 + rawDataSize 160 serialization.ddl struct pcr_t1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -769,13 +766,13 @@ STAGE PLANS: Select Operator expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string), KEY.reducesinkkey1 (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 13 Data size: 104 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 13 Data size: 104 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -891,20 +888,20 @@ STAGE PLANS: Map Operator Tree: TableScan alias: pcr_t1 - Statistics: Num rows: 2 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: (((ds < '2000-04-09') and (key < 5)) or ((ds > '2000-04-09') and (value = 'val_5'))) (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 10 Data size: 80 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int), value (type: string), ds (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 10 Data size: 80 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int), _col2 (type: string) sort order: ++ - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 10 Data size: 80 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: string) auto parallelism: false @@ -927,10 +924,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t1 numFiles 1 - numRows -1 + numRows 20 partition_columns ds partition_columns.types string - rawDataSize -1 + rawDataSize 160 serialization.ddl struct pcr_t1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -972,10 +969,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t1 numFiles 1 - numRows -1 + numRows 20 partition_columns ds partition_columns.types string - rawDataSize -1 + rawDataSize 160 serialization.ddl struct pcr_t1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1010,13 +1007,13 @@ STAGE PLANS: Select Operator expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string), KEY.reducesinkkey1 (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 10 Data size: 80 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 10 Data size: 80 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -1134,20 +1131,20 @@ STAGE PLANS: Map Operator Tree: TableScan alias: pcr_t1 - Statistics: Num rows: 3 Data size: 540 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 60 Data size: 480 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: (((ds < '2000-04-10') and (key < 5)) or ((ds > '2000-04-08') and (value = 'val_5'))) (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 16 Data size: 128 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int), value (type: string), ds (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 16 Data size: 128 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int), _col2 (type: string) sort order: ++ - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 16 Data size: 128 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: string) auto parallelism: false @@ -1170,10 +1167,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t1 numFiles 1 - numRows -1 + numRows 20 partition_columns ds partition_columns.types string - rawDataSize -1 + rawDataSize 160 serialization.ddl struct pcr_t1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1215,10 +1212,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t1 numFiles 1 - numRows -1 + numRows 20 partition_columns ds partition_columns.types string - rawDataSize -1 + rawDataSize 160 serialization.ddl struct pcr_t1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1260,10 +1257,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t1 numFiles 1 - numRows -1 + numRows 20 partition_columns ds partition_columns.types string - rawDataSize -1 + rawDataSize 160 serialization.ddl struct pcr_t1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1299,13 +1296,13 @@ STAGE PLANS: Select Operator expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string), KEY.reducesinkkey1 (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 16 Data size: 128 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 16 Data size: 128 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -1433,20 +1430,20 @@ STAGE PLANS: Map Operator Tree: TableScan alias: pcr_t1 - Statistics: Num rows: 3 Data size: 540 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 60 Data size: 480 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: (((ds < '2000-04-10') or (key < 5)) and ((ds > '2000-04-08') or (value = 'val_5'))) (type: boolean) - Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 33 Data size: 264 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int), value (type: string), ds (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 33 Data size: 264 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int), _col2 (type: string) sort order: ++ - Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 33 Data size: 264 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: string) auto parallelism: false @@ -1469,10 +1466,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t1 numFiles 1 - numRows -1 + numRows 20 partition_columns ds partition_columns.types string - rawDataSize -1 + rawDataSize 160 serialization.ddl struct pcr_t1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1514,10 +1511,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t1 numFiles 1 - numRows -1 + numRows 20 partition_columns ds partition_columns.types string - rawDataSize -1 + rawDataSize 160 serialization.ddl struct pcr_t1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1559,10 +1556,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t1 numFiles 1 - numRows -1 + numRows 20 partition_columns ds partition_columns.types string - rawDataSize -1 + rawDataSize 160 serialization.ddl struct pcr_t1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1598,13 +1595,13 @@ STAGE PLANS: Select Operator expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string), KEY.reducesinkkey1 (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 33 Data size: 264 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 33 Data size: 264 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -1732,20 +1729,20 @@ STAGE PLANS: Map Operator Tree: TableScan alias: pcr_t1 - Statistics: Num rows: 2 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: (key = 14) (type: boolean) - Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: value (type: string) outputColumnNames: _col1 - Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: 14 (type: int), _col1 (type: string) sort order: ++ - Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE tag: -1 auto parallelism: false Path -> Alias: @@ -1767,10 +1764,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t1 numFiles 1 - numRows -1 + numRows 20 partition_columns ds partition_columns.types string - rawDataSize -1 + rawDataSize 160 serialization.ddl struct pcr_t1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1812,10 +1809,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t1 numFiles 1 - numRows -1 + numRows 20 partition_columns ds partition_columns.types string - rawDataSize -1 + rawDataSize 160 serialization.ddl struct pcr_t1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1850,13 +1847,13 @@ STAGE PLANS: Select Operator expressions: 14 (type: int), KEY.reducesinkkey1 (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -1947,16 +1944,16 @@ STAGE PLANS: Map Operator Tree: TableScan alias: pcr_t1 - Statistics: Num rows: 2 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE GatherStats: false Select Operator expressions: key (type: int), value (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 2 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int), _col1 (type: string) sort order: ++ - Statistics: Num rows: 2 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE tag: -1 auto parallelism: false Path -> Alias: @@ -1978,10 +1975,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t1 numFiles 1 - numRows -1 + numRows 20 partition_columns ds partition_columns.types string - rawDataSize -1 + rawDataSize 160 serialization.ddl struct pcr_t1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2023,10 +2020,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t1 numFiles 1 - numRows -1 + numRows 20 partition_columns ds partition_columns.types string - rawDataSize -1 + rawDataSize 160 serialization.ddl struct pcr_t1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2061,13 +2058,13 @@ STAGE PLANS: Select Operator expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 2 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 2 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -2198,16 +2195,16 @@ STAGE PLANS: Map Operator Tree: TableScan alias: pcr_t1 - Statistics: Num rows: 3 Data size: 540 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 60 Data size: 480 Basic stats: COMPLETE Column stats: NONE GatherStats: false Select Operator expressions: key (type: int), value (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 3 Data size: 540 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 60 Data size: 480 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int), _col1 (type: string) sort order: ++ - Statistics: Num rows: 3 Data size: 540 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 60 Data size: 480 Basic stats: COMPLETE Column stats: NONE tag: -1 auto parallelism: false Path -> Alias: @@ -2229,10 +2226,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t1 numFiles 1 - numRows -1 + numRows 20 partition_columns ds partition_columns.types string - rawDataSize -1 + rawDataSize 160 serialization.ddl struct pcr_t1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2274,10 +2271,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t1 numFiles 1 - numRows -1 + numRows 20 partition_columns ds partition_columns.types string - rawDataSize -1 + rawDataSize 160 serialization.ddl struct pcr_t1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2319,10 +2316,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t1 numFiles 1 - numRows -1 + numRows 20 partition_columns ds partition_columns.types string - rawDataSize -1 + rawDataSize 160 serialization.ddl struct pcr_t1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2358,13 +2355,13 @@ STAGE PLANS: Select Operator expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 3 Data size: 540 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 60 Data size: 480 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 3 Data size: 540 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 60 Data size: 480 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -2533,20 +2530,20 @@ STAGE PLANS: Map Operator Tree: TableScan alias: pcr_t1 - Statistics: Num rows: 2 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: (((ds = '2000-04-08') and (key = 1)) or ((ds = '2000-04-09') and (key = 2))) (type: boolean) - Statistics: Num rows: 2 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int), value (type: string), ds (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 2 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string) sort order: +++ - Statistics: Num rows: 2 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE tag: -1 auto parallelism: false Path -> Alias: @@ -2568,10 +2565,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t1 numFiles 1 - numRows -1 + numRows 20 partition_columns ds partition_columns.types string - rawDataSize -1 + rawDataSize 160 serialization.ddl struct pcr_t1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2613,10 +2610,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t1 numFiles 1 - numRows -1 + numRows 20 partition_columns ds partition_columns.types string - rawDataSize -1 + rawDataSize 160 serialization.ddl struct pcr_t1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2651,13 +2648,13 @@ STAGE PLANS: Select Operator expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 2 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 2 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -2764,17 +2761,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t2 - Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: key is not null (type: boolean) - Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 80 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 80 Basic stats: COMPLETE Column stats: NONE tag: 1 value expressions: value (type: string), ds (type: string) auto parallelism: true @@ -2797,10 +2794,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t1 numFiles 1 - numRows -1 + numRows 20 partition_columns ds partition_columns.types string - rawDataSize -1 + rawDataSize 160 serialization.ddl struct pcr_t1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2832,17 +2829,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t1 - Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: key is not null (type: boolean) - Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 80 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 80 Basic stats: COMPLETE Column stats: NONE tag: 0 value expressions: value (type: string), ds (type: string) auto parallelism: true @@ -2865,10 +2862,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t1 numFiles 1 - numRows -1 + numRows 20 partition_columns ds partition_columns.types string - rawDataSize -1 + rawDataSize 160 serialization.ddl struct pcr_t1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2906,15 +2903,15 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} {VALUE._col1} 1 {KEY.reducesinkkey0} {VALUE._col0} {VALUE._col1} outputColumnNames: _col0, _col1, _col2, _col6, _col7, _col8 - Statistics: Num rows: 1 Data size: 198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 88 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col6 (type: int), _col7 (type: string), _col8 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 1 Data size: 198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 88 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) sort order: + - Statistics: Num rows: 1 Data size: 198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 88 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: int), _col4 (type: string), _col5 (type: string) auto parallelism: false @@ -2924,13 +2921,13 @@ STAGE PLANS: Select Operator expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: string), VALUE._col4 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 1 Data size: 198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 88 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 1 Data size: 198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 88 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -3072,17 +3069,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t2 - Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: key is not null (type: boolean) - Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 80 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 80 Basic stats: COMPLETE Column stats: NONE tag: 1 value expressions: value (type: string), ds (type: string) auto parallelism: true @@ -3105,10 +3102,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t1 numFiles 1 - numRows -1 + numRows 20 partition_columns ds partition_columns.types string - rawDataSize -1 + rawDataSize 160 serialization.ddl struct pcr_t1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -3140,17 +3137,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t1 - Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: key is not null (type: boolean) - Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 80 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 80 Basic stats: COMPLETE Column stats: NONE tag: 0 value expressions: value (type: string), ds (type: string) auto parallelism: true @@ -3173,10 +3170,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t1 numFiles 1 - numRows -1 + numRows 20 partition_columns ds partition_columns.types string - rawDataSize -1 + rawDataSize 160 serialization.ddl struct pcr_t1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -3214,15 +3211,15 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} {VALUE._col1} 1 {KEY.reducesinkkey0} {VALUE._col0} {VALUE._col1} outputColumnNames: _col0, _col1, _col2, _col6, _col7, _col8 - Statistics: Num rows: 1 Data size: 198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 88 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col6 (type: int), _col7 (type: string), _col8 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 1 Data size: 198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 88 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) sort order: + - Statistics: Num rows: 1 Data size: 198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 88 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: int), _col4 (type: string), _col5 (type: string) auto parallelism: false @@ -3232,13 +3229,13 @@ STAGE PLANS: Select Operator expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: string), VALUE._col4 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 1 Data size: 198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 88 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 1 Data size: 198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 88 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -3315,7 +3312,6 @@ PREHOOK: query: insert overwrite table pcr_t1 partition (ds='2000-04-11') select PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@pcr_t1@ds=2000-04-11 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table pcr_t1 partition (ds='2000-04-11') select * from src where key < 20 order by key POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -3399,20 +3395,20 @@ STAGE PLANS: Map Operator Tree: TableScan alias: pcr_t1 - Statistics: Num rows: 4 Data size: 720 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 80 Data size: 640 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: (((ds > '2000-04-08') and (ds < '2000-04-11')) or (key = 2)) (type: boolean) - Statistics: Num rows: 2 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 48 Data size: 384 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int), value (type: string), ds (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 2 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 48 Data size: 384 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string) sort order: +++ - Statistics: Num rows: 2 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 48 Data size: 384 Basic stats: COMPLETE Column stats: NONE tag: -1 auto parallelism: false Path -> Alias: @@ -3434,10 +3430,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t1 numFiles 1 - numRows -1 + numRows 20 partition_columns ds partition_columns.types string - rawDataSize -1 + rawDataSize 160 serialization.ddl struct pcr_t1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -3479,10 +3475,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t1 numFiles 1 - numRows -1 + numRows 20 partition_columns ds partition_columns.types string - rawDataSize -1 + rawDataSize 160 serialization.ddl struct pcr_t1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -3524,10 +3520,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t1 numFiles 1 - numRows -1 + numRows 20 partition_columns ds partition_columns.types string - rawDataSize -1 + rawDataSize 160 serialization.ddl struct pcr_t1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -3569,10 +3565,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t1 numFiles 1 - numRows -1 + numRows 20 partition_columns ds partition_columns.types string - rawDataSize -1 + rawDataSize 160 serialization.ddl struct pcr_t1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -3609,13 +3605,13 @@ STAGE PLANS: Select Operator expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 2 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 48 Data size: 384 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 2 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 48 Data size: 384 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -3768,20 +3764,20 @@ STAGE PLANS: Map Operator Tree: TableScan alias: pcr_t1 - Statistics: Num rows: 3 Data size: 540 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 60 Data size: 480 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: ((ds > '2000-04-08') or ((ds <= '2000-04-09') and (key = 2))) (type: boolean) - Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 30 Data size: 240 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int), value (type: string), ds (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 30 Data size: 240 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string) sort order: +++ - Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 30 Data size: 240 Basic stats: COMPLETE Column stats: NONE tag: -1 auto parallelism: false Path -> Alias: @@ -3803,10 +3799,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t1 numFiles 1 - numRows -1 + numRows 20 partition_columns ds partition_columns.types string - rawDataSize -1 + rawDataSize 160 serialization.ddl struct pcr_t1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -3848,10 +3844,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t1 numFiles 1 - numRows -1 + numRows 20 partition_columns ds partition_columns.types string - rawDataSize -1 + rawDataSize 160 serialization.ddl struct pcr_t1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -3893,10 +3889,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t1 numFiles 1 - numRows -1 + numRows 20 partition_columns ds partition_columns.types string - rawDataSize -1 + rawDataSize 160 serialization.ddl struct pcr_t1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -3932,13 +3928,13 @@ STAGE PLANS: Select Operator expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 30 Data size: 240 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 30 Data size: 240 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -4102,18 +4098,18 @@ STAGE PLANS: Map Operator Tree: TableScan alias: pcr_t1 - Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE GatherStats: false Select Operator expressions: key (type: int), value (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 1 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -4137,13 +4133,13 @@ STAGE PLANS: Select Operator expressions: key (type: int), value (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 2 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -4183,10 +4179,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t1 numFiles 1 - numRows -1 + numRows 20 partition_columns ds partition_columns.types string - rawDataSize -1 + rawDataSize 160 serialization.ddl struct pcr_t1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -4278,8 +4274,6 @@ PREHOOK: Input: default@pcr_t1 PREHOOK: Input: default@pcr_t1@ds=2000-04-08 PREHOOK: Output: default@pcr_t2 PREHOOK: Output: default@pcr_t3 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: from pcr_t1 insert overwrite table pcr_t2 select key, value where ds='2000-04-08' insert overwrite table pcr_t3 select key, value where ds='2000-04-08' @@ -4372,22 +4366,22 @@ STAGE PLANS: Map Operator Tree: TableScan alias: pcr_t1 - Statistics: Num rows: 1 Data size: 180 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: (key = 2) (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 10 Data size: 80 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: 2 (type: int), value (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 10 Data size: 80 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 1 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 10 Data size: 80 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -4401,8 +4395,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t2 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 0 + rawDataSize 0 serialization.ddl struct pcr_t2 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -4416,17 +4410,17 @@ STAGE PLANS: Filter Operator isSamplingPred: false predicate: (key = 3) (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 10 Data size: 80 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: 3 (type: int), value (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 10 Data size: 80 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 2 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 10 Data size: 80 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -4440,8 +4434,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t3 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 0 + rawDataSize 0 serialization.ddl struct pcr_t3 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -4471,10 +4465,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t1 numFiles 1 - numRows -1 + numRows 20 partition_columns ds partition_columns.types string - rawDataSize -1 + rawDataSize 160 serialization.ddl struct pcr_t1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -4523,8 +4517,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t2 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 0 + rawDataSize 0 serialization.ddl struct pcr_t2 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -4554,8 +4548,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.pcr_t3 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 0 + rawDataSize 0 serialization.ddl struct pcr_t3 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -4576,8 +4570,6 @@ PREHOOK: Input: default@pcr_t1 PREHOOK: Input: default@pcr_t1@ds=2000-04-08 PREHOOK: Output: default@pcr_t2 PREHOOK: Output: default@pcr_t3 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: from pcr_t1 insert overwrite table pcr_t2 select key, value where ds='2000-04-08' and key=2 insert overwrite table pcr_t3 select key, value where ds='2000-04-08' and key=3 @@ -5265,7 +5257,6 @@ PREHOOK: query: insert overwrite table pcr_foo partition (ds=3) select * from sr PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@pcr_foo@ds=3 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table pcr_foo partition (ds=3) select * from src where key < 10 order by key POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -5276,7 +5267,6 @@ PREHOOK: query: insert overwrite table pcr_foo partition (ds=5) select * from sr PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@pcr_foo@ds=5 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table pcr_foo partition (ds=5) select * from src where key < 10 order by key POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -5287,7 +5277,6 @@ PREHOOK: query: insert overwrite table pcr_foo partition (ds=7) select * from sr PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@pcr_foo@ds=7 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table pcr_foo partition (ds=7) select * from src where key < 10 order by key POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -5428,7 +5417,6 @@ PREHOOK: query: insert overwrite table foo_field partition (ds=5) select strct f PREHOOK: type: QUERY PREHOOK: Input: default@ab PREHOOK: Output: default@foo_field@ds=5 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table foo_field partition (ds=5) select strct from ab where strct.a < 10 limit 2 POSTHOOK: type: QUERY POSTHOOK: Input: default@ab @@ -5438,7 +5426,6 @@ PREHOOK: query: insert overwrite table foo_field partition (ds=7) select strct f PREHOOK: type: QUERY PREHOOK: Input: default@ab PREHOOK: Output: default@foo_field@ds=7 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table foo_field partition (ds=7) select strct from ab where strct.a > 190 limit 2 POSTHOOK: type: QUERY POSTHOOK: Input: default@ab diff --git ql/src/test/results/clientpositive/spark/sample1.q.out ql/src/test/results/clientpositive/spark/sample1.q.out index e6c5e62..71837d3 100644 --- ql/src/test/results/clientpositive/spark/sample1.q.out +++ ql/src/test/results/clientpositive/spark/sample1.q.out @@ -196,7 +196,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@srcpart PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 PREHOOK: Output: default@dest1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE dest1 SELECT s.* FROM srcpart TABLESAMPLE (BUCKET 1 OUT OF 1 ON rand()) s WHERE s.ds='2008-04-08' and s.hr='11' diff --git ql/src/test/results/clientpositive/spark/sample10.q.out ql/src/test/results/clientpositive/spark/sample10.q.out index c511152..ebda402 100644 --- ql/src/test/results/clientpositive/spark/sample10.q.out +++ ql/src/test/results/clientpositive/spark/sample10.q.out @@ -18,7 +18,6 @@ PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 PREHOOK: Output: default@srcpartbucket -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table srcpartbucket partition(ds, hr) select * from srcpart where ds is not null and key < 10 POSTHOOK: type: QUERY POSTHOOK: Input: default@srcpart @@ -98,27 +97,27 @@ STAGE PLANS: Map Operator Tree: TableScan alias: srcpartbucket - Statistics: Num rows: 12 Data size: 1404 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 240 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: true predicate: (((hash(key) & 2147483647) % 4) = 0) (type: boolean) - Statistics: Num rows: 6 Data size: 702 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 120 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: ds (type: string) outputColumnNames: ds - Statistics: Num rows: 6 Data size: 702 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 120 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) keys: ds (type: string) mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 6 Data size: 702 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 120 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) sort order: + Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 6 Data size: 702 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 120 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: bigint) auto parallelism: true @@ -143,10 +142,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.srcpartbucket numFiles 4 - numRows -1 + numRows 10 partition_columns ds/hr partition_columns.types string:string - rawDataSize -1 + rawDataSize 60 serialization.ddl struct srcpartbucket { string key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe @@ -191,10 +190,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.srcpartbucket numFiles 4 - numRows -1 + numRows 10 partition_columns ds/hr partition_columns.types string:string - rawDataSize -1 + rawDataSize 60 serialization.ddl struct srcpartbucket { string key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe @@ -239,10 +238,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.srcpartbucket numFiles 4 - numRows -1 + numRows 10 partition_columns ds/hr partition_columns.types string:string - rawDataSize -1 + rawDataSize 60 serialization.ddl struct srcpartbucket { string key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe @@ -287,10 +286,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.srcpartbucket numFiles 4 - numRows -1 + numRows 10 partition_columns ds/hr partition_columns.types string:string - rawDataSize -1 + rawDataSize 60 serialization.ddl struct srcpartbucket { string key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe @@ -330,15 +329,15 @@ STAGE PLANS: keys: KEY._col0 (type: string) mode: mergepartial outputColumnNames: _col0, _col1 - Statistics: Num rows: 3 Data size: 351 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 60 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col1 (type: bigint) outputColumnNames: _col0, _col1 - Statistics: Num rows: 3 Data size: 351 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 60 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) sort order: + - Statistics: Num rows: 3 Data size: 351 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 60 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: bigint) auto parallelism: false @@ -348,13 +347,13 @@ STAGE PLANS: Select Operator expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: bigint) outputColumnNames: _col0, _col1 - Statistics: Num rows: 3 Data size: 351 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 60 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 3 Data size: 351 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 60 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat diff --git ql/src/test/results/clientpositive/spark/sample2.q.out ql/src/test/results/clientpositive/spark/sample2.q.out index 2b4fcd6..a7b5067 100644 --- ql/src/test/results/clientpositive/spark/sample2.q.out +++ ql/src/test/results/clientpositive/spark/sample2.q.out @@ -178,7 +178,6 @@ FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 2) s PREHOOK: type: QUERY PREHOOK: Input: default@srcbucket PREHOOK: Output: default@dest1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE dest1 SELECT s.* FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 2) s POSTHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/spark/sample4.q.out ql/src/test/results/clientpositive/spark/sample4.q.out index 6b12cd6..d1fcc3b 100644 --- ql/src/test/results/clientpositive/spark/sample4.q.out +++ ql/src/test/results/clientpositive/spark/sample4.q.out @@ -180,7 +180,6 @@ FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 2 on key) s PREHOOK: type: QUERY PREHOOK: Input: default@srcbucket PREHOOK: Output: default@dest1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE dest1 SELECT s.* FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 2 on key) s POSTHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/spark/sample5.q.out ql/src/test/results/clientpositive/spark/sample5.q.out index a605c92..14fab3f 100644 --- ql/src/test/results/clientpositive/spark/sample5.q.out +++ ql/src/test/results/clientpositive/spark/sample5.q.out @@ -178,7 +178,6 @@ FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 5 on key) s PREHOOK: type: QUERY PREHOOK: Input: default@srcbucket PREHOOK: Output: default@dest1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE dest1 SELECT s.* -- here's another test FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 5 on key) s POSTHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/spark/sample6.q.out ql/src/test/results/clientpositive/spark/sample6.q.out index f6256f5..8445610 100644 --- ql/src/test/results/clientpositive/spark/sample6.q.out +++ ql/src/test/results/clientpositive/spark/sample6.q.out @@ -178,7 +178,6 @@ FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 4 on key) s PREHOOK: type: QUERY PREHOOK: Input: default@srcbucket PREHOOK: Output: default@dest1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE dest1 SELECT s.* FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 4 on key) s POSTHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/spark/sample7.q.out ql/src/test/results/clientpositive/spark/sample7.q.out index f99d898..6bf11a6 100644 --- ql/src/test/results/clientpositive/spark/sample7.q.out +++ ql/src/test/results/clientpositive/spark/sample7.q.out @@ -188,7 +188,6 @@ WHERE s.key > 100 PREHOOK: type: QUERY PREHOOK: Input: default@srcbucket PREHOOK: Output: default@dest1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE dest1 SELECT s.* FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 4 on key) s WHERE s.key > 100 diff --git ql/src/test/results/clientpositive/spark/scriptfile1.q.out ql/src/test/results/clientpositive/spark/scriptfile1.q.out index 5d91e90..c8d120c 100644 --- ql/src/test/results/clientpositive/spark/scriptfile1.q.out +++ ql/src/test/results/clientpositive/spark/scriptfile1.q.out @@ -18,7 +18,6 @@ INSERT OVERWRITE TABLE dest1 SELECT tmap.tkey, tmap.tvalue PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM ( FROM src SELECT TRANSFORM(src.key, src.value) diff --git ql/src/test/results/clientpositive/spark/skewjoin.q.out ql/src/test/results/clientpositive/spark/skewjoin.q.out index d674d04..f2645c6 100644 --- ql/src/test/results/clientpositive/spark/skewjoin.q.out +++ ql/src/test/results/clientpositive/spark/skewjoin.q.out @@ -162,7 +162,6 @@ INSERT OVERWRITE TABLE dest_j1 SELECT src1.key, src2.value PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest_j1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM src src1 JOIN src src2 ON (src1.key = src2.key) INSERT OVERWRITE TABLE dest_j1 SELECT src1.key, src2.value POSTHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/spark/smb_mapjoin_13.q.out ql/src/test/results/clientpositive/spark/smb_mapjoin_13.q.out index 0319137..d928aa2 100644 --- ql/src/test/results/clientpositive/spark/smb_mapjoin_13.q.out +++ ql/src/test/results/clientpositive/spark/smb_mapjoin_13.q.out @@ -47,10 +47,6 @@ PREHOOK: Output: default@test_table1 PREHOOK: Output: default@test_table2 PREHOOK: Output: default@test_table3 PREHOOK: Output: default@test_table4 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM src INSERT OVERWRITE TABLE test_table1 SELECT * INSERT OVERWRITE TABLE test_table2 SELECT * @@ -173,8 +169,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.test_table2 numFiles 16 - numRows -1 - rawDataSize -1 + numRows 0 + rawDataSize 0 serialization.ddl struct test_table2 { i32 value, string key} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -195,8 +191,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.test_table2 numFiles 16 - numRows -1 - rawDataSize -1 + numRows 0 + rawDataSize 0 serialization.ddl struct test_table2 { i32 value, string key} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -211,17 +207,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: a - Statistics: Num rows: 55 Data size: 5812 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: key is not null (type: boolean) - Statistics: Num rows: 28 Data size: 2958 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - Statistics: Num rows: 28 Data size: 2958 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE tag: 0 value expressions: value (type: string) auto parallelism: true @@ -244,8 +240,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.test_table1 numFiles 16 - numRows -1 - rawDataSize -1 + numRows 500 + rawDataSize 5312 serialization.ddl struct test_table1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -266,8 +262,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.test_table1 numFiles 16 - numRows -1 - rawDataSize -1 + numRows 500 + rawDataSize 5312 serialization.ddl struct test_table1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -288,15 +284,15 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 {KEY.reducesinkkey0} {VALUE._col0} outputColumnNames: _col0, _col1, _col5, _col6 - Statistics: Num rows: 30 Data size: 3253 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col5 (type: int), _col6 (type: string) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 30 Data size: 3253 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) sort order: + - Statistics: Num rows: 30 Data size: 3253 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: string) auto parallelism: false @@ -306,16 +302,16 @@ STAGE PLANS: Select Operator expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string), VALUE._col1 (type: int), VALUE._col2 (type: string) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 30 Data size: 3253 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE Limit Number of rows: 10 - Statistics: Num rows: 10 Data size: 1080 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 100 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 10 Data size: 1080 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 100 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -461,8 +457,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.test_table4 numFiles 16 - numRows -1 - rawDataSize -1 + numRows 0 + rawDataSize 0 serialization.ddl struct test_table4 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -483,8 +479,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.test_table4 numFiles 16 - numRows -1 - rawDataSize -1 + numRows 0 + rawDataSize 0 serialization.ddl struct test_table4 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -532,8 +528,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.test_table3 numFiles 16 - numRows -1 - rawDataSize -1 + numRows 0 + rawDataSize 0 serialization.ddl struct test_table3 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -554,8 +550,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.test_table3 numFiles 16 - numRows -1 - rawDataSize -1 + numRows 0 + rawDataSize 0 serialization.ddl struct test_table3 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe diff --git ql/src/test/results/clientpositive/spark/smb_mapjoin_15.q.out ql/src/test/results/clientpositive/spark/smb_mapjoin_15.q.out index 7849e78..65c9f18 100644 --- ql/src/test/results/clientpositive/spark/smb_mapjoin_15.q.out +++ ql/src/test/results/clientpositive/spark/smb_mapjoin_15.q.out @@ -29,8 +29,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@test_table1 PREHOOK: Output: default@test_table2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM src INSERT OVERWRITE TABLE test_table1 SELECT * INSERT OVERWRITE TABLE test_table2 SELECT * @@ -143,8 +141,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.test_table2 numFiles 16 - numRows -1 - rawDataSize -1 + numRows 0 + rawDataSize 0 serialization.ddl struct test_table2 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -165,8 +163,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.test_table2 numFiles 16 - numRows -1 - rawDataSize -1 + numRows 0 + rawDataSize 0 serialization.ddl struct test_table2 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -181,17 +179,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: a - Statistics: Num rows: 55 Data size: 5812 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: key is not null (type: boolean) - Statistics: Num rows: 28 Data size: 2958 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - Statistics: Num rows: 28 Data size: 2958 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE tag: 0 value expressions: value (type: string) auto parallelism: true @@ -214,8 +212,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.test_table1 numFiles 16 - numRows -1 - rawDataSize -1 + numRows 500 + rawDataSize 5312 serialization.ddl struct test_table1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -236,8 +234,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.test_table1 numFiles 16 - numRows -1 - rawDataSize -1 + numRows 500 + rawDataSize 5312 serialization.ddl struct test_table1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -258,15 +256,15 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 {KEY.reducesinkkey0} {VALUE._col0} outputColumnNames: _col0, _col1, _col5, _col6 - Statistics: Num rows: 30 Data size: 3253 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col5 (type: int), _col6 (type: string) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 30 Data size: 3253 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) sort order: + - Statistics: Num rows: 30 Data size: 3253 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: string), _col2 (type: int), _col3 (type: string) auto parallelism: false @@ -276,16 +274,16 @@ STAGE PLANS: Select Operator expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string), VALUE._col1 (type: int), VALUE._col2 (type: string) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 30 Data size: 3253 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE Limit Number of rows: 10 - Statistics: Num rows: 10 Data size: 1080 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 100 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 10 Data size: 1080 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 100 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -369,8 +367,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@test_table1 PREHOOK: Output: default@test_table2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM src INSERT OVERWRITE TABLE test_table1 SELECT key, key, value INSERT OVERWRITE TABLE test_table2 SELECT key, key, value @@ -495,8 +491,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.test_table2 numFiles 16 - numRows -1 - rawDataSize -1 + numRows 0 + rawDataSize 0 serialization.ddl struct test_table2 { i32 key, i32 key2, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -517,8 +513,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.test_table2 numFiles 16 - numRows -1 - rawDataSize -1 + numRows 0 + rawDataSize 0 serialization.ddl struct test_table2 { i32 key, i32 key2, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -533,17 +529,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: a - Statistics: Num rows: 71 Data size: 7718 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 500 Data size: 7218 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: (key is not null and key2 is not null) (type: boolean) - Statistics: Num rows: 18 Data size: 1956 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 125 Data size: 1804 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: int), key2 (type: int) sort order: ++ Map-reduce partition columns: key (type: int), key2 (type: int) - Statistics: Num rows: 18 Data size: 1956 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 125 Data size: 1804 Basic stats: COMPLETE Column stats: NONE tag: 0 value expressions: value (type: string) auto parallelism: true @@ -566,8 +562,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.test_table1 numFiles 16 - numRows -1 - rawDataSize -1 + numRows 500 + rawDataSize 7218 serialization.ddl struct test_table1 { i32 key, i32 key2, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -588,8 +584,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.test_table1 numFiles 16 - numRows -1 - rawDataSize -1 + numRows 500 + rawDataSize 7218 serialization.ddl struct test_table1 { i32 key, i32 key2, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -610,15 +606,15 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {KEY.reducesinkkey1} {VALUE._col0} 1 {KEY.reducesinkkey0} {KEY.reducesinkkey1} {VALUE._col0} outputColumnNames: _col0, _col1, _col2, _col6, _col7, _col8 - Statistics: Num rows: 19 Data size: 2151 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 137 Data size: 1984 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string), _col6 (type: int), _col7 (type: int), _col8 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 19 Data size: 2151 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 137 Data size: 1984 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) sort order: + - Statistics: Num rows: 19 Data size: 2151 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 137 Data size: 1984 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: int), _col2 (type: string), _col3 (type: int), _col4 (type: int), _col5 (type: string) auto parallelism: false @@ -628,16 +624,16 @@ STAGE PLANS: Select Operator expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 19 Data size: 2151 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 137 Data size: 1984 Basic stats: COMPLETE Column stats: NONE Limit Number of rows: 10 - Statistics: Num rows: 10 Data size: 1130 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 140 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 10 Data size: 1130 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 140 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -793,8 +789,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.test_table2 numFiles 16 - numRows -1 - rawDataSize -1 + numRows 0 + rawDataSize 0 serialization.ddl struct test_table2 { i32 key, i32 key2, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -815,8 +811,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.test_table2 numFiles 16 - numRows -1 - rawDataSize -1 + numRows 0 + rawDataSize 0 serialization.ddl struct test_table2 { i32 key, i32 key2, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -831,17 +827,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: a - Statistics: Num rows: 71 Data size: 7718 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 500 Data size: 7218 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: (key2 is not null and key is not null) (type: boolean) - Statistics: Num rows: 18 Data size: 1956 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 125 Data size: 1804 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key2 (type: int), key (type: int) sort order: ++ Map-reduce partition columns: key2 (type: int), key (type: int) - Statistics: Num rows: 18 Data size: 1956 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 125 Data size: 1804 Basic stats: COMPLETE Column stats: NONE tag: 0 value expressions: value (type: string) auto parallelism: true @@ -864,8 +860,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.test_table1 numFiles 16 - numRows -1 - rawDataSize -1 + numRows 500 + rawDataSize 7218 serialization.ddl struct test_table1 { i32 key, i32 key2, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -886,8 +882,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.test_table1 numFiles 16 - numRows -1 - rawDataSize -1 + numRows 500 + rawDataSize 7218 serialization.ddl struct test_table1 { i32 key, i32 key2, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -908,15 +904,15 @@ STAGE PLANS: 0 {KEY.reducesinkkey1} {KEY.reducesinkkey0} {VALUE._col0} 1 {KEY.reducesinkkey1} {KEY.reducesinkkey0} {VALUE._col0} outputColumnNames: _col0, _col1, _col2, _col6, _col7, _col8 - Statistics: Num rows: 19 Data size: 2151 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 137 Data size: 1984 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string), _col6 (type: int), _col7 (type: int), _col8 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 19 Data size: 2151 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 137 Data size: 1984 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) sort order: + - Statistics: Num rows: 19 Data size: 2151 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 137 Data size: 1984 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: int), _col2 (type: string), _col3 (type: int), _col4 (type: int), _col5 (type: string) auto parallelism: false @@ -926,16 +922,16 @@ STAGE PLANS: Select Operator expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 19 Data size: 2151 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 137 Data size: 1984 Basic stats: COMPLETE Column stats: NONE Limit Number of rows: 10 - Statistics: Num rows: 10 Data size: 1130 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 140 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 10 Data size: 1130 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 140 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -1091,8 +1087,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.test_table2 numFiles 16 - numRows -1 - rawDataSize -1 + numRows 0 + rawDataSize 0 serialization.ddl struct test_table2 { i32 key, i32 key2, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1113,8 +1109,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.test_table2 numFiles 16 - numRows -1 - rawDataSize -1 + numRows 0 + rawDataSize 0 serialization.ddl struct test_table2 { i32 key, i32 key2, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1129,17 +1125,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: a - Statistics: Num rows: 71 Data size: 7718 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 500 Data size: 7218 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: (key is not null and value is not null) (type: boolean) - Statistics: Num rows: 18 Data size: 1956 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 125 Data size: 1804 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: int), value (type: string) sort order: ++ Map-reduce partition columns: key (type: int), value (type: string) - Statistics: Num rows: 18 Data size: 1956 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 125 Data size: 1804 Basic stats: COMPLETE Column stats: NONE tag: 0 value expressions: key2 (type: int) auto parallelism: true @@ -1162,8 +1158,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.test_table1 numFiles 16 - numRows -1 - rawDataSize -1 + numRows 500 + rawDataSize 7218 serialization.ddl struct test_table1 { i32 key, i32 key2, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1184,8 +1180,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.test_table1 numFiles 16 - numRows -1 - rawDataSize -1 + numRows 500 + rawDataSize 7218 serialization.ddl struct test_table1 { i32 key, i32 key2, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1206,15 +1202,15 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} {KEY.reducesinkkey1} 1 {KEY.reducesinkkey0} {VALUE._col0} {KEY.reducesinkkey1} outputColumnNames: _col0, _col1, _col2, _col6, _col7, _col8 - Statistics: Num rows: 19 Data size: 2151 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 137 Data size: 1984 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string), _col6 (type: int), _col7 (type: int), _col8 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 19 Data size: 2151 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 137 Data size: 1984 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) sort order: + - Statistics: Num rows: 19 Data size: 2151 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 137 Data size: 1984 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: int), _col2 (type: string), _col3 (type: int), _col4 (type: int), _col5 (type: string) auto parallelism: false @@ -1224,16 +1220,16 @@ STAGE PLANS: Select Operator expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 19 Data size: 2151 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 137 Data size: 1984 Basic stats: COMPLETE Column stats: NONE Limit Number of rows: 10 - Statistics: Num rows: 10 Data size: 1130 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 140 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 10 Data size: 1130 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 140 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat diff --git ql/src/test/results/clientpositive/spark/smb_mapjoin_16.q.out ql/src/test/results/clientpositive/spark/smb_mapjoin_16.q.out index 11ffefd..e1f4fc5 100644 --- ql/src/test/results/clientpositive/spark/smb_mapjoin_16.q.out +++ ql/src/test/results/clientpositive/spark/smb_mapjoin_16.q.out @@ -23,8 +23,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@test_table1 PREHOOK: Output: default@test_table2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM src INSERT OVERWRITE TABLE test_table1 SELECT * INSERT OVERWRITE TABLE test_table2 SELECT * @@ -73,15 +71,15 @@ STAGE PLANS: Map Operator Tree: TableScan alias: a - Statistics: Num rows: 1453 Data size: 5812 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 727 Data size: 2908 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - Statistics: Num rows: 727 Data size: 2908 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator diff --git ql/src/test/results/clientpositive/spark/subquery_multiinsert.q.out ql/src/test/results/clientpositive/spark/subquery_multiinsert.q.out index 2bedd37..f825e82 100644 --- ql/src/test/results/clientpositive/spark/subquery_multiinsert.q.out +++ ql/src/test/results/clientpositive/spark/subquery_multiinsert.q.out @@ -289,8 +289,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@src_4 PREHOOK: Output: default@src_5 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: from src b INSERT OVERWRITE TABLE src_4 select * @@ -724,8 +722,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@src_4 PREHOOK: Output: default@src_5 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: from src b INSERT OVERWRITE TABLE src_4 select * diff --git ql/src/test/results/clientpositive/spark/table_access_keys_stats.q.out ql/src/test/results/clientpositive/spark/table_access_keys_stats.q.out index 844d36f..fbe392e 100644 --- ql/src/test/results/clientpositive/spark/table_access_keys_stats.q.out +++ ql/src/test/results/clientpositive/spark/table_access_keys_stats.q.out @@ -260,8 +260,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@t1 PREHOOK: Output: default@t2 PREHOOK: Output: default@t3 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to Operator:GBY_2 Table:default@t1 Keys:key diff --git ql/src/test/results/clientpositive/spark/timestamp_1.q.out ql/src/test/results/clientpositive/spark/timestamp_1.q.out index 2d48500..d3ca5cf 100644 --- ql/src/test/results/clientpositive/spark/timestamp_1.q.out +++ ql/src/test/results/clientpositive/spark/timestamp_1.q.out @@ -23,7 +23,6 @@ PREHOOK: query: insert overwrite table timestamp_1 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@timestamp_1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table timestamp_1 select cast('2011-01-01 01:01:01' as timestamp) from src tablesample (1 rows) POSTHOOK: type: QUERY @@ -107,7 +106,6 @@ PREHOOK: query: insert overwrite table timestamp_1 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@timestamp_1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table timestamp_1 select '2011-01-01 01:01:01' from src tablesample (1 rows) POSTHOOK: type: QUERY @@ -191,7 +189,6 @@ PREHOOK: query: insert overwrite table timestamp_1 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@timestamp_1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table timestamp_1 select '2011-01-01 01:01:01.1' from src tablesample (1 rows) POSTHOOK: type: QUERY @@ -275,7 +272,6 @@ PREHOOK: query: insert overwrite table timestamp_1 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@timestamp_1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table timestamp_1 select '2011-01-01 01:01:01.0001' from src tablesample (1 rows) POSTHOOK: type: QUERY @@ -359,7 +355,6 @@ PREHOOK: query: insert overwrite table timestamp_1 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@timestamp_1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table timestamp_1 select '2011-01-01 01:01:01.000100000' from src tablesample (1 rows) POSTHOOK: type: QUERY @@ -443,7 +438,6 @@ PREHOOK: query: insert overwrite table timestamp_1 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@timestamp_1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table timestamp_1 select '2011-01-01 01:01:01.001000011' from src tablesample (1 rows) POSTHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/spark/timestamp_2.q.out ql/src/test/results/clientpositive/spark/timestamp_2.q.out index f41ecbe..f9bfb09 100644 --- ql/src/test/results/clientpositive/spark/timestamp_2.q.out +++ ql/src/test/results/clientpositive/spark/timestamp_2.q.out @@ -23,7 +23,6 @@ PREHOOK: query: insert overwrite table timestamp_2 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@timestamp_2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table timestamp_2 select cast('2011-01-01 01:01:01' as timestamp) from src tablesample (1 rows) POSTHOOK: type: QUERY @@ -107,7 +106,6 @@ PREHOOK: query: insert overwrite table timestamp_2 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@timestamp_2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table timestamp_2 select '2011-01-01 01:01:01' from src tablesample (1 rows) POSTHOOK: type: QUERY @@ -191,7 +189,6 @@ PREHOOK: query: insert overwrite table timestamp_2 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@timestamp_2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table timestamp_2 select '2011-01-01 01:01:01.1' from src tablesample (1 rows) POSTHOOK: type: QUERY @@ -275,7 +272,6 @@ PREHOOK: query: insert overwrite table timestamp_2 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@timestamp_2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table timestamp_2 select '2011-01-01 01:01:01.0001' from src tablesample (1 rows) POSTHOOK: type: QUERY @@ -359,7 +355,6 @@ PREHOOK: query: insert overwrite table timestamp_2 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@timestamp_2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table timestamp_2 select '2011-01-01 01:01:01.000100000' from src tablesample (1 rows) POSTHOOK: type: QUERY @@ -443,7 +438,6 @@ PREHOOK: query: insert overwrite table timestamp_2 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@timestamp_2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table timestamp_2 select '2011-01-01 01:01:01.001000011' from src tablesample (1 rows) POSTHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/spark/timestamp_3.q.out ql/src/test/results/clientpositive/spark/timestamp_3.q.out index 7bb6b19..0664abf 100644 --- ql/src/test/results/clientpositive/spark/timestamp_3.q.out +++ ql/src/test/results/clientpositive/spark/timestamp_3.q.out @@ -23,7 +23,6 @@ PREHOOK: query: insert overwrite table timestamp_3 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@timestamp_3 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table timestamp_3 select cast(cast('1.3041352164485E9' as double) as timestamp) from src tablesample (1 rows) POSTHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/spark/timestamp_lazy.q.out ql/src/test/results/clientpositive/spark/timestamp_lazy.q.out index 640ca6b..6ac8cba 100644 --- ql/src/test/results/clientpositive/spark/timestamp_lazy.q.out +++ ql/src/test/results/clientpositive/spark/timestamp_lazy.q.out @@ -14,7 +14,6 @@ PREHOOK: query: insert overwrite table timestamp_lazy select cast('2011-01-01 01 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@timestamp_lazy -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table timestamp_lazy select cast('2011-01-01 01:01:01' as timestamp), key, value from src tablesample (5 rows) POSTHOOK: type: QUERY POSTHOOK: Input: default@src diff --git ql/src/test/results/clientpositive/spark/timestamp_udf.q.out ql/src/test/results/clientpositive/spark/timestamp_udf.q.out index 4cfca4c..904c894 100644 --- ql/src/test/results/clientpositive/spark/timestamp_udf.q.out +++ ql/src/test/results/clientpositive/spark/timestamp_udf.q.out @@ -31,8 +31,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@timestamp_udf PREHOOK: Output: default@timestamp_udf_string -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: from (select * from src tablesample (1 rows)) s insert overwrite table timestamp_udf select '2011-05-06 07:08:09.1234567' diff --git ql/src/test/results/clientpositive/spark/transform1.q.out ql/src/test/results/clientpositive/spark/transform1.q.out index a657d01..481f922 100644 --- ql/src/test/results/clientpositive/spark/transform1.q.out +++ ql/src/test/results/clientpositive/spark/transform1.q.out @@ -72,7 +72,6 @@ select array(1,2,3) from src tablesample (1 rows) PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@transform1_t2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table transform1_t2 select array(1,2,3) from src tablesample (1 rows) POSTHOOK: type: QUERY @@ -98,21 +97,21 @@ STAGE PLANS: Map Operator Tree: TableScan alias: transform1_t2 - Statistics: Num rows: -1 Data size: 6 Basic stats: PARTIAL Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: '012' (type: string) outputColumnNames: _col0 - Statistics: Num rows: -1 Data size: 6 Basic stats: PARTIAL Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 89 Basic stats: COMPLETE Column stats: COMPLETE Transform Operator command: cat output info: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Statistics: Num rows: -1 Data size: 6 Basic stats: PARTIAL Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 89 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: -1 Data size: 6 Basic stats: PARTIAL Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 89 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat diff --git ql/src/test/results/clientpositive/spark/union10.q.out ql/src/test/results/clientpositive/spark/union10.q.out index 59ebb0c..1e58d04 100644 --- ql/src/test/results/clientpositive/spark/union10.q.out +++ ql/src/test/results/clientpositive/spark/union10.q.out @@ -175,7 +175,6 @@ PREHOOK: query: insert overwrite table tmptable PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@tmptable -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table tmptable select unionsrc.key, unionsrc.value FROM (select 'tst1' as key, count(1) as value from src s1 UNION ALL diff --git ql/src/test/results/clientpositive/spark/union18.q.out ql/src/test/results/clientpositive/spark/union18.q.out index f1c69bf..617a005 100644 --- ql/src/test/results/clientpositive/spark/union18.q.out +++ ql/src/test/results/clientpositive/spark/union18.q.out @@ -161,8 +161,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 PREHOOK: Output: default@dest2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM (select 'tst1' as key, cast(count(1) as string) as value from src s1 UNION ALL select s2.key as key, s2.value as value from src s2) unionsrc diff --git ql/src/test/results/clientpositive/spark/union19.q.out ql/src/test/results/clientpositive/spark/union19.q.out index c86afb0..aa1eea3 100644 --- ql/src/test/results/clientpositive/spark/union19.q.out +++ ql/src/test/results/clientpositive/spark/union19.q.out @@ -188,8 +188,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@dest1 PREHOOK: Output: default@dest2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM (select 'tst1' as key, cast(count(1) as string) as value from src s1 UNION ALL select s2.key as key, s2.value as value from src s2) unionsrc diff --git ql/src/test/results/clientpositive/spark/union25.q.out ql/src/test/results/clientpositive/spark/union25.q.out index 07ba875..4a36b29 100644 --- ql/src/test/results/clientpositive/spark/union25.q.out +++ ql/src/test/results/clientpositive/spark/union25.q.out @@ -12,7 +12,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@srcpart PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 PREHOOK: Output: default@tmp_srcpart@ds=2008-04-08/hr=11 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table tmp_srcpart partition (ds='2008-04-08', hr='11') select key, value from srcpart where ds='2008-04-08' and hr='11' POSTHOOK: type: QUERY @@ -159,14 +158,14 @@ STAGE PLANS: keys: KEY._col0 (type: string), KEY._col1 (type: string) mode: mergepartial outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 279 Data size: 5562 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col2 (type: bigint), _col0 (type: string), _col1 (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 279 Data size: 5562 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 279 Data size: 5562 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat diff --git ql/src/test/results/clientpositive/spark/union28.q.out ql/src/test/results/clientpositive/spark/union28.q.out index f668ff8..8d1beae 100644 --- ql/src/test/results/clientpositive/spark/union28.q.out +++ ql/src/test/results/clientpositive/spark/union28.q.out @@ -185,7 +185,6 @@ select * from ( PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@union_subq_union -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table union_subq_union select * from ( select key, value from src diff --git ql/src/test/results/clientpositive/spark/union29.q.out ql/src/test/results/clientpositive/spark/union29.q.out index f82d561..56d2723 100644 --- ql/src/test/results/clientpositive/spark/union29.q.out +++ ql/src/test/results/clientpositive/spark/union29.q.out @@ -135,7 +135,6 @@ select * from ( PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@union_subq_union -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table union_subq_union select * from ( select key, value from src diff --git ql/src/test/results/clientpositive/spark/union3.q.out ql/src/test/results/clientpositive/spark/union3.q.out index ba21367..7782965 100644 --- ql/src/test/results/clientpositive/spark/union3.q.out +++ ql/src/test/results/clientpositive/spark/union3.q.out @@ -238,7 +238,6 @@ FROM ( PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@union_out -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table union_out SELECT * FROM ( diff --git ql/src/test/results/clientpositive/spark/union30.q.out ql/src/test/results/clientpositive/spark/union30.q.out index ee0daf4..51d4a1f 100644 --- ql/src/test/results/clientpositive/spark/union30.q.out +++ ql/src/test/results/clientpositive/spark/union30.q.out @@ -232,7 +232,6 @@ select key, value from src PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@union_subq_union -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table union_subq_union select * from ( diff --git ql/src/test/results/clientpositive/spark/union33.q.out ql/src/test/results/clientpositive/spark/union33.q.out index ca08e0c..7ac8dea 100644 --- ql/src/test/results/clientpositive/spark/union33.q.out +++ ql/src/test/results/clientpositive/spark/union33.q.out @@ -151,7 +151,6 @@ UNION ALL PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@test_src -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE test_src SELECT key, value FROM ( SELECT key, value FROM src @@ -311,7 +310,6 @@ UNION ALL PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@test_src -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE test_src SELECT key, value FROM ( SELECT key, COUNT(*) AS value FROM src diff --git ql/src/test/results/clientpositive/spark/union4.q.out ql/src/test/results/clientpositive/spark/union4.q.out index 2e46204..ea232f4 100644 --- ql/src/test/results/clientpositive/spark/union4.q.out +++ ql/src/test/results/clientpositive/spark/union4.q.out @@ -135,7 +135,6 @@ select unionsrc.key, unionsrc.value FROM (select 'tst1' as key, count(1) as valu PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@tmptable -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table tmptable select unionsrc.key, unionsrc.value FROM (select 'tst1' as key, count(1) as value from src s1 UNION ALL diff --git ql/src/test/results/clientpositive/spark/union6.q.out ql/src/test/results/clientpositive/spark/union6.q.out index 01f044e..c166e1b 100644 --- ql/src/test/results/clientpositive/spark/union6.q.out +++ ql/src/test/results/clientpositive/spark/union6.q.out @@ -117,7 +117,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Input: default@src1 PREHOOK: Output: default@tmptable -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table tmptable select unionsrc.key, unionsrc.value FROM (select 'tst1' as key, cast(count(1) as string) as value from src s1 UNION ALL diff --git ql/src/test/results/clientpositive/spark/vector_cast_constant.q.out ql/src/test/results/clientpositive/spark/vector_cast_constant.q.out index 2dd7aab..60ab429 100644 --- ql/src/test/results/clientpositive/spark/vector_cast_constant.q.out +++ ql/src/test/results/clientpositive/spark/vector_cast_constant.q.out @@ -82,7 +82,6 @@ PREHOOK: query: INSERT INTO TABLE over1korc SELECT * FROM over1k PREHOOK: type: QUERY PREHOOK: Input: default@over1k PREHOOK: Output: default@over1korc -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT INTO TABLE over1korc SELECT * FROM over1k POSTHOOK: type: QUERY POSTHOOK: Input: default@over1k @@ -128,22 +127,22 @@ STAGE PLANS: Map Operator Tree: TableScan alias: over1korc - Statistics: Num rows: 5898 Data size: 23594 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1049 Data size: 311170 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: i (type: int) outputColumnNames: i - Statistics: Num rows: 5898 Data size: 23594 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1049 Data size: 311170 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: avg(50), avg(UDFToDouble(50)), avg(CAST( 50 AS decimal(10,0))) keys: i (type: int) mode: hash outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 5898 Data size: 23594 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1049 Data size: 311170 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 5898 Data size: 23594 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1049 Data size: 311170 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: struct), _col2 (type: struct), _col3 (type: struct) Execution mode: vectorized Reducer 2 @@ -153,28 +152,28 @@ STAGE PLANS: keys: KEY._col0 (type: int) mode: mergepartial outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 2949 Data size: 11797 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 524 Data size: 155436 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: double), _col2 (type: double), _col3 (type: decimal(14,4)) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 2949 Data size: 11797 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 524 Data size: 155436 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) sort order: + - Statistics: Num rows: 2949 Data size: 11797 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 524 Data size: 155436 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: double), _col2 (type: double), _col3 (type: decimal(14,4)) Reducer 3 Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: double), VALUE._col1 (type: double), VALUE._col2 (type: decimal(14,4)) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 2949 Data size: 11797 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 524 Data size: 155436 Basic stats: COMPLETE Column stats: NONE Limit Number of rows: 10 - Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 2960 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 2960 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat diff --git ql/src/test/results/clientpositive/spark/vector_char_4.q.out ql/src/test/results/clientpositive/spark/vector_char_4.q.out index f0d0a1a..b0c2670 100644 --- ql/src/test/results/clientpositive/spark/vector_char_4.q.out +++ ql/src/test/results/clientpositive/spark/vector_char_4.q.out @@ -92,7 +92,6 @@ PREHOOK: query: INSERT INTO TABLE vectortab2korc SELECT * FROM vectortab2k PREHOOK: type: QUERY PREHOOK: Input: default@vectortab2k PREHOOK: Output: default@vectortab2korc -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT INTO TABLE vectortab2korc SELECT * FROM vectortab2k POSTHOOK: type: QUERY POSTHOOK: Input: default@vectortab2k @@ -143,14 +142,14 @@ STAGE PLANS: Map Operator Tree: TableScan alias: vectortab2korc - Statistics: Num rows: 731 Data size: 96518 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: CAST( t AS CHAR(10) (type: char(10)), CAST( si AS CHAR(10) (type: char(10)), CAST( i AS CHAR(20) (type: char(20)), CAST( b AS CHAR(30) (type: char(30)), CAST( f AS CHAR(20) (type: char(20)), CAST( d AS CHAR(20) (type: char(20)), CAST( s AS CHAR(50) (type: char(50)) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 731 Data size: 96518 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 731 Data size: 96518 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat diff --git ql/src/test/results/clientpositive/spark/vector_count_distinct.q.out ql/src/test/results/clientpositive/spark/vector_count_distinct.q.out index 8b6a226..19d45e1 100644 --- ql/src/test/results/clientpositive/spark/vector_count_distinct.q.out +++ ql/src/test/results/clientpositive/spark/vector_count_distinct.q.out @@ -197,7 +197,6 @@ select ws_sold_date_sk, ws_sold_time_sk, ws_ship_date_sk, ws_item_sk, PREHOOK: type: QUERY PREHOOK: Input: default@web_sales_txt PREHOOK: Output: default@web_sales -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table web_sales partition (ws_web_site_sk) select ws_sold_date_sk, ws_sold_time_sk, ws_ship_date_sk, ws_item_sk, @@ -1255,21 +1254,21 @@ STAGE PLANS: Map Operator Tree: TableScan alias: web_sales - Statistics: Num rows: 58981 Data size: 235968 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2000 Data size: 3504000 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: ws_order_number (type: int) outputColumnNames: ws_order_number - Statistics: Num rows: 58981 Data size: 235968 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2000 Data size: 3504000 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(DISTINCT ws_order_number) keys: ws_order_number (type: int) mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 58981 Data size: 235968 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2000 Data size: 3504000 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) sort order: + - Statistics: Num rows: 58981 Data size: 235968 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2000 Data size: 3504000 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Reducer 2 Reduce Operator Tree: diff --git ql/src/test/results/clientpositive/spark/vector_data_types.q.out ql/src/test/results/clientpositive/spark/vector_data_types.q.out index 5758c4b..1cce6c8 100644 --- ql/src/test/results/clientpositive/spark/vector_data_types.q.out +++ ql/src/test/results/clientpositive/spark/vector_data_types.q.out @@ -82,7 +82,6 @@ PREHOOK: query: INSERT INTO TABLE over1korc SELECT * FROM over1k PREHOOK: type: QUERY PREHOOK: Input: default@over1k PREHOOK: Output: default@over1korc -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT INTO TABLE over1korc SELECT * FROM over1k POSTHOOK: type: QUERY POSTHOOK: Input: default@over1k @@ -117,28 +116,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: over1korc - Statistics: Num rows: 60 Data size: 23594 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1049 Data size: 311170 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: t (type: tinyint), si (type: smallint), i (type: int), b (type: bigint), f (type: float), d (type: double), bo (type: boolean), s (type: string), ts (type: timestamp), dec (type: decimal(4,2)), bin (type: binary) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10 - Statistics: Num rows: 60 Data size: 23594 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1049 Data size: 311170 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col2 (type: int) sort order: + - Statistics: Num rows: 60 Data size: 23594 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1049 Data size: 311170 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: boolean), _col7 (type: string), _col8 (type: timestamp), _col9 (type: decimal(4,2)), _col10 (type: binary) Reducer 2 Reduce Operator Tree: Select Operator expressions: VALUE._col0 (type: tinyint), VALUE._col1 (type: smallint), KEY.reducesinkkey0 (type: int), VALUE._col2 (type: bigint), VALUE._col3 (type: float), VALUE._col4 (type: double), VALUE._col5 (type: boolean), VALUE._col6 (type: string), VALUE._col7 (type: timestamp), VALUE._col8 (type: decimal(4,2)), VALUE._col9 (type: binary) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10 - Statistics: Num rows: 60 Data size: 23594 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1049 Data size: 311170 Basic stats: COMPLETE Column stats: NONE Limit Number of rows: 20 - Statistics: Num rows: 20 Data size: 7860 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 5920 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 20 Data size: 7860 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 5920 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -197,15 +196,15 @@ STAGE PLANS: Map Operator Tree: TableScan alias: over1korc - Statistics: Num rows: 60 Data size: 23594 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1049 Data size: 311170 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: t (type: tinyint), si (type: smallint), i (type: int), b (type: bigint), f (type: float), d (type: double), bo (type: boolean), s (type: string), ts (type: timestamp), dec (type: decimal(4,2)), bin (type: binary) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10 - Statistics: Num rows: 60 Data size: 23594 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1049 Data size: 311170 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col2 (type: int) sort order: + - Statistics: Num rows: 60 Data size: 23594 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1049 Data size: 311170 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: boolean), _col7 (type: string), _col8 (type: timestamp), _col9 (type: decimal(4,2)), _col10 (type: binary) Execution mode: vectorized Reducer 2 @@ -213,13 +212,13 @@ STAGE PLANS: Select Operator expressions: VALUE._col0 (type: tinyint), VALUE._col1 (type: smallint), KEY.reducesinkkey0 (type: int), VALUE._col2 (type: bigint), VALUE._col3 (type: float), VALUE._col4 (type: double), VALUE._col5 (type: boolean), VALUE._col6 (type: string), VALUE._col7 (type: timestamp), VALUE._col8 (type: decimal(4,2)), VALUE._col9 (type: binary) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10 - Statistics: Num rows: 60 Data size: 23594 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1049 Data size: 311170 Basic stats: COMPLETE Column stats: NONE Limit Number of rows: 20 - Statistics: Num rows: 20 Data size: 7860 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 5920 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 20 Data size: 7860 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 5920 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat diff --git ql/src/test/results/clientpositive/spark/vector_orderby_5.q.out ql/src/test/results/clientpositive/spark/vector_orderby_5.q.out index 3b89885..8122e10 100644 --- ql/src/test/results/clientpositive/spark/vector_orderby_5.q.out +++ ql/src/test/results/clientpositive/spark/vector_orderby_5.q.out @@ -84,7 +84,6 @@ PREHOOK: query: INSERT INTO TABLE vectortab2korc SELECT * FROM vectortab2k PREHOOK: type: QUERY PREHOOK: Input: default@vectortab2k PREHOOK: Output: default@vectortab2korc -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT INTO TABLE vectortab2korc SELECT * FROM vectortab2k POSTHOOK: type: QUERY POSTHOOK: Input: default@vectortab2k @@ -124,22 +123,22 @@ STAGE PLANS: Map Operator Tree: TableScan alias: vectortab2korc - Statistics: Num rows: 8043 Data size: 96518 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: bo (type: boolean), b (type: bigint) outputColumnNames: bo, b - Statistics: Num rows: 8043 Data size: 96518 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: max(b) keys: bo (type: boolean) mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 8043 Data size: 96518 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: boolean) sort order: + Map-reduce partition columns: _col0 (type: boolean) - Statistics: Num rows: 8043 Data size: 96518 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Execution mode: vectorized Reducer 2 @@ -149,15 +148,15 @@ STAGE PLANS: keys: KEY._col0 (type: boolean) mode: mergepartial outputColumnNames: _col0, _col1 - Statistics: Num rows: 4021 Data size: 48252 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1000 Data size: 459356 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: boolean), _col1 (type: bigint) outputColumnNames: _col0, _col1 - Statistics: Num rows: 4021 Data size: 48252 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1000 Data size: 459356 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: boolean) sort order: - - Statistics: Num rows: 4021 Data size: 48252 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1000 Data size: 459356 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Execution mode: vectorized Reducer 3 @@ -165,10 +164,10 @@ STAGE PLANS: Select Operator expressions: KEY.reducesinkkey0 (type: boolean), VALUE._col0 (type: bigint) outputColumnNames: _col0, _col1 - Statistics: Num rows: 4021 Data size: 48252 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1000 Data size: 459356 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 4021 Data size: 48252 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1000 Data size: 459356 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat diff --git ql/src/test/results/clientpositive/spark/vector_string_concat.q.out ql/src/test/results/clientpositive/spark/vector_string_concat.q.out index b45f042..56e7e10 100644 --- ql/src/test/results/clientpositive/spark/vector_string_concat.q.out +++ ql/src/test/results/clientpositive/spark/vector_string_concat.q.out @@ -82,7 +82,6 @@ PREHOOK: query: INSERT INTO TABLE over1korc SELECT * FROM over1k PREHOOK: type: QUERY PREHOOK: Input: default@over1k PREHOOK: Output: default@over1korc -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT INTO TABLE over1korc SELECT * FROM over1k POSTHOOK: type: QUERY POSTHOOK: Input: default@over1k @@ -118,14 +117,14 @@ STAGE PLANS: Processor Tree: TableScan alias: over1korc - Statistics: Num rows: 235 Data size: 23594 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1049 Data size: 311170 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: s (type: string), concat(concat(' ', s), ' ') (type: string), concat(concat('|', rtrim(concat(concat(' ', s), ' '))), '|') (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 235 Data size: 23594 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1049 Data size: 311170 Basic stats: COMPLETE Column stats: NONE Limit Number of rows: 20 - Statistics: Num rows: 20 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 5920 Basic stats: COMPLETE Column stats: NONE ListSink PREHOOK: query: SELECT s AS `string`, diff --git ql/src/test/results/clientpositive/spark/vector_varchar_4.q.out ql/src/test/results/clientpositive/spark/vector_varchar_4.q.out index 37be6bc..b552cd2 100644 --- ql/src/test/results/clientpositive/spark/vector_varchar_4.q.out +++ ql/src/test/results/clientpositive/spark/vector_varchar_4.q.out @@ -92,7 +92,6 @@ PREHOOK: query: INSERT INTO TABLE vectortab2korc SELECT * FROM vectortab2k PREHOOK: type: QUERY PREHOOK: Input: default@vectortab2k PREHOOK: Output: default@vectortab2korc -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT INTO TABLE vectortab2korc SELECT * FROM vectortab2k POSTHOOK: type: QUERY POSTHOOK: Input: default@vectortab2k @@ -143,14 +142,14 @@ STAGE PLANS: Map Operator Tree: TableScan alias: vectortab2korc - Statistics: Num rows: 731 Data size: 96518 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: CAST( t AS varchar(10)) (type: varchar(10)), CAST( si AS varchar(10)) (type: varchar(10)), CAST( i AS varchar(20)) (type: varchar(20)), CAST( b AS varchar(30)) (type: varchar(30)), CAST( f AS varchar(20)) (type: varchar(20)), CAST( d AS varchar(20)) (type: varchar(20)), CAST( s AS varchar(50)) (type: varchar(50)) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 731 Data size: 96518 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 731 Data size: 96518 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2000 Data size: 918712 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat diff --git ql/src/test/results/clientpositive/spark/vectorization_part.q.out ql/src/test/results/clientpositive/spark/vectorization_part.q.out index 44e43c1..66facc9 100644 --- ql/src/test/results/clientpositive/spark/vectorization_part.q.out +++ ql/src/test/results/clientpositive/spark/vectorization_part.q.out @@ -10,7 +10,6 @@ PREHOOK: query: insert overwrite table alltypesorc_part partition (ds='2011') se PREHOOK: type: QUERY PREHOOK: Input: default@alltypesorc PREHOOK: Output: default@alltypesorc_part@ds=2011 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table alltypesorc_part partition (ds='2011') select * from alltypesorc limit 100 POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc @@ -31,7 +30,6 @@ PREHOOK: query: insert overwrite table alltypesorc_part partition (ds='2012') se PREHOOK: type: QUERY PREHOOK: Input: default@alltypesorc PREHOOK: Output: default@alltypesorc_part@ds=2012 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table alltypesorc_part partition (ds='2012') select * from alltypesorc limit 100 POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc diff --git ql/src/test/results/clientpositive/spark/vectorization_part_project.q.out ql/src/test/results/clientpositive/spark/vectorization_part_project.q.out index aa87dd9..5f4f3bb 100644 --- ql/src/test/results/clientpositive/spark/vectorization_part_project.q.out +++ ql/src/test/results/clientpositive/spark/vectorization_part_project.q.out @@ -10,7 +10,6 @@ PREHOOK: query: insert overwrite table alltypesorc_part partition (ds='2011') se PREHOOK: type: QUERY PREHOOK: Input: default@alltypesorc PREHOOK: Output: default@alltypesorc_part@ds=2011 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table alltypesorc_part partition (ds='2011') select * from alltypesorc limit 100 POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc @@ -31,7 +30,6 @@ PREHOOK: query: insert overwrite table alltypesorc_part partition (ds='2012') se PREHOOK: type: QUERY PREHOOK: Input: default@alltypesorc PREHOOK: Output: default@alltypesorc_part@ds=2012 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table alltypesorc_part partition (ds='2012') select * from alltypesorc limit 100 POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc @@ -67,28 +65,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: alltypesorc_part - Statistics: Num rows: 706 Data size: 5648 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 200 Data size: 41576 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: (cdouble + 2) (type: double) outputColumnNames: _col0 - Statistics: Num rows: 706 Data size: 5648 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 200 Data size: 41576 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: double) sort order: + - Statistics: Num rows: 706 Data size: 5648 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 200 Data size: 41576 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Reducer 2 Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey0 (type: double) outputColumnNames: _col0 - Statistics: Num rows: 706 Data size: 5648 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 200 Data size: 41576 Basic stats: COMPLETE Column stats: NONE Limit Number of rows: 10 - Statistics: Num rows: 10 Data size: 80 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 2070 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 10 Data size: 80 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 2070 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat diff --git ql/src/test/results/clientpositive/spark/vectorized_bucketmapjoin1.q.out ql/src/test/results/clientpositive/spark/vectorized_bucketmapjoin1.q.out index 17d8235..354a7ff 100644 --- ql/src/test/results/clientpositive/spark/vectorized_bucketmapjoin1.q.out +++ ql/src/test/results/clientpositive/spark/vectorized_bucketmapjoin1.q.out @@ -58,7 +58,6 @@ PREHOOK: query: insert into table vsmb_bucket_1 select cint, cstring1 from allty PREHOOK: type: QUERY PREHOOK: Input: default@alltypesorc PREHOOK: Output: default@vsmb_bucket_1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert into table vsmb_bucket_1 select cint, cstring1 from alltypesorc limit 2 POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc @@ -69,7 +68,6 @@ PREHOOK: query: insert into table vsmb_bucket_2 select cint, cstring1 from allty PREHOOK: type: QUERY PREHOOK: Input: default@alltypesorc PREHOOK: Output: default@vsmb_bucket_2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert into table vsmb_bucket_2 select cint, cstring1 from alltypesorc limit 2 POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc @@ -80,7 +78,6 @@ PREHOOK: query: insert into table vsmb_bucket_RC select cint, cstring1 from allt PREHOOK: type: QUERY PREHOOK: Input: default@alltypesorc PREHOOK: Output: default@vsmb_bucket_rc -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert into table vsmb_bucket_RC select cint, cstring1 from alltypesorc limit 2 POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc @@ -91,7 +88,6 @@ PREHOOK: query: insert into table vsmb_bucket_TXT select cint, cstring1 from all PREHOOK: type: QUERY PREHOOK: Input: default@alltypesorc PREHOOK: Output: default@vsmb_bucket_txt -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert into table vsmb_bucket_TXT select cint, cstring1 from alltypesorc limit 2 POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc @@ -119,30 +115,30 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 3 Data size: 369 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 208 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 2 Data size: 246 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 104 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - Statistics: Num rows: 2 Data size: 246 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 104 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Execution mode: vectorized Map 3 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 3 Data size: 369 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 208 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 2 Data size: 246 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 104 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - Statistics: Num rows: 2 Data size: 246 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 104 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Execution mode: vectorized Reducer 2 @@ -154,14 +150,14 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 {KEY.reducesinkkey0} {VALUE._col0} outputColumnNames: _col0, _col1, _col5, _col6 - Statistics: Num rows: 2 Data size: 270 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col5 (type: int), _col6 (type: string) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 2 Data size: 270 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 2 Data size: 270 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -208,29 +204,29 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 1 Data size: 129 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 50 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 1 Data size: 129 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 25 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - Statistics: Num rows: 1 Data size: 129 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 25 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Map 3 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 3 Data size: 369 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 208 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 2 Data size: 246 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 104 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - Statistics: Num rows: 2 Data size: 246 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 104 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Execution mode: vectorized Reducer 2 @@ -242,14 +238,14 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 {KEY.reducesinkkey0} {VALUE._col0} outputColumnNames: _col0, _col1, _col5, _col6 - Statistics: Num rows: 2 Data size: 270 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col5 (type: int), _col6 (type: string) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 2 Data size: 270 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 2 Data size: 270 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -306,29 +302,29 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 0 Data size: 54 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 2 Data size: 52 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 26 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 26 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Map 3 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 3 Data size: 369 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 208 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 2 Data size: 246 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 104 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - Statistics: Num rows: 2 Data size: 246 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 104 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Execution mode: vectorized Reducer 2 @@ -340,14 +336,14 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 {KEY.reducesinkkey0} {VALUE._col0} outputColumnNames: _col0, _col1, _col5, _col6 - Statistics: Num rows: 2 Data size: 270 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col5 (type: int), _col6 (type: string) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 2 Data size: 270 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 2 Data size: 270 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat diff --git ql/src/test/results/clientpositive/spark/vectorized_ptf.q.out ql/src/test/results/clientpositive/spark/vectorized_ptf.q.out index 1d11b30..0742475 100644 --- ql/src/test/results/clientpositive/spark/vectorized_ptf.q.out +++ ql/src/test/results/clientpositive/spark/vectorized_ptf.q.out @@ -101,7 +101,6 @@ PREHOOK: query: insert into table part select * from part_staging PREHOOK: type: QUERY PREHOOK: Input: default@part_staging PREHOOK: Output: default@part -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert into table part select * from part_staging POSTHOOK: type: QUERY POSTHOOK: Input: default@part_staging @@ -233,13 +232,13 @@ STAGE PLANS: Map Operator Tree: TableScan alias: part - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE GatherStats: false Reduce Output Operator key expressions: p_mfgr (type: string), p_name (type: string) sort order: ++ Map-reduce partition columns: p_mfgr (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true @@ -260,6 +259,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -278,6 +279,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -292,18 +295,18 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int), _col7 (type: double) outputColumnNames: _col1, _col2, _col5, _col7 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col2 (type: string), _col1 (type: string) sort order: ++ Map-reduce partition columns: _col2 (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int), _col7 (type: double) auto parallelism: true @@ -311,19 +314,19 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col2 (type: string), _col1 (type: string), _col5 (type: int), _wcol0 (type: int), _wcol1 (type: int), _wcol2 (type: double) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -515,17 +518,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: p1 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: p_partkey is not null (type: boolean) - Statistics: Num rows: 2 Data size: 1298 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: p_partkey (type: int) sort order: + Map-reduce partition columns: p_partkey (type: int) - Statistics: Num rows: 2 Data size: 1298 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE tag: 0 value expressions: p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string) auto parallelism: true @@ -546,6 +549,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -564,6 +569,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -579,17 +586,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: p2 - Statistics: Num rows: 649 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: p_partkey is not null (type: boolean) - Statistics: Num rows: 325 Data size: 1300 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: p_partkey (type: int) sort order: + Map-reduce partition columns: p_partkey (type: int) - Statistics: Num rows: 325 Data size: 1300 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE tag: 1 auto parallelism: true Path -> Alias: @@ -609,6 +616,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -627,6 +636,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -648,16 +659,16 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} {VALUE._col1} {VALUE._col2} {VALUE._col3} {VALUE._col4} {VALUE._col5} {VALUE._col6} {VALUE._col7} 1 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 357 Data size: 1430 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 14 Data size: 8823 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 357 Data size: 1430 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 14 Data size: 8823 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col2 (type: string), _col1 (type: string) sort order: ++ Map-reduce partition columns: _col2 (type: string) - Statistics: Num rows: 357 Data size: 1430 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 14 Data size: 8823 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) auto parallelism: true @@ -665,18 +676,18 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 357 Data size: 1430 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 14 Data size: 8823 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 357 Data size: 1430 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 14 Data size: 8823 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int) outputColumnNames: _col1, _col2, _col5 - Statistics: Num rows: 357 Data size: 1430 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 14 Data size: 8823 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col2 (type: string), _col1 (type: string) sort order: ++ Map-reduce partition columns: _col2 (type: string) - Statistics: Num rows: 357 Data size: 1430 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 14 Data size: 8823 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int) auto parallelism: true @@ -684,19 +695,19 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 357 Data size: 1430 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 14 Data size: 8823 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 357 Data size: 1430 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 14 Data size: 8823 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col2 (type: string), _col1 (type: string), _col5 (type: int), (_col5 - _wcol0) (type: int) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 357 Data size: 1430 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 14 Data size: 8823 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 357 Data size: 1430 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 14 Data size: 8823 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -827,13 +838,13 @@ STAGE PLANS: Map Operator Tree: TableScan alias: part - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE GatherStats: false Reduce Output Operator key expressions: p_mfgr (type: string), p_name (type: string) sort order: ++ Map-reduce partition columns: p_mfgr (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true @@ -854,6 +865,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -872,6 +885,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -886,19 +901,19 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col2 (type: string), _col1 (type: string), _col5 (type: int) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -1080,13 +1095,13 @@ STAGE PLANS: Map Operator Tree: TableScan alias: part - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE GatherStats: false Reduce Output Operator key expressions: p_mfgr (type: string), p_name (type: string) sort order: ++ Map-reduce partition columns: p_mfgr (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true @@ -1107,6 +1122,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -1125,6 +1142,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -1139,18 +1158,18 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int), _col7 (type: double) outputColumnNames: _col1, _col2, _col5, _col7 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col2 (type: string), _col1 (type: string) sort order: ++ Map-reduce partition columns: _col2 (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int), _col7 (type: double) auto parallelism: true @@ -1158,19 +1177,19 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col2 (type: string), _col1 (type: string), _col5 (type: int), _wcol0 (type: int), _wcol1 (type: int), _wcol2 (type: double) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -1364,13 +1383,13 @@ STAGE PLANS: Map Operator Tree: TableScan alias: part - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE GatherStats: false Reduce Output Operator key expressions: p_mfgr (type: string), p_name (type: string) sort order: ++ Map-reduce partition columns: p_mfgr (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true @@ -1391,6 +1410,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -1409,6 +1430,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -1423,18 +1446,18 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int) outputColumnNames: _col1, _col2, _col5 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col2 (type: string), _col1 (type: string) sort order: ++ Map-reduce partition columns: _col2 (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int) auto parallelism: true @@ -1442,19 +1465,19 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col2 (type: string), _col1 (type: string), _col5 (type: int), _wcol0 (type: int), _wcol1 (type: int), _col5 (type: int), (_col5 - _wcol2) (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -1658,13 +1681,13 @@ STAGE PLANS: Map Operator Tree: TableScan alias: part - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE GatherStats: false Reduce Output Operator key expressions: p_mfgr (type: string), p_name (type: string) sort order: ++ Map-reduce partition columns: p_mfgr (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true @@ -1685,6 +1708,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -1703,6 +1728,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -1717,23 +1744,23 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col2 (type: string), _col1 (type: string), _col5 (type: int) outputColumnNames: _col2, _col1, _col5 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: _col2 (type: string), _col1 (type: string), _col5 (type: int) mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: int) sort order: +++ Map-reduce partition columns: _col0 (type: string), _col1 (type: string), _col2 (type: int) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 auto parallelism: true Reducer 3 @@ -1743,12 +1770,12 @@ STAGE PLANS: keys: KEY._col0 (type: string), KEY._col1 (type: string), KEY._col2 (type: int) mode: mergepartial outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string), _col1 (type: string) sort order: ++ Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: int) auto parallelism: true @@ -1757,19 +1784,19 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col1 (type: string), _col2 (type: int), _wcol0 (type: int), _wcol1 (type: int), _col2 (type: int), (_col2 - _wcol2) (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -1918,17 +1945,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: p1 - Statistics: Num rows: 649 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: p_partkey is not null (type: boolean) - Statistics: Num rows: 325 Data size: 1300 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: p_partkey (type: int) sort order: + Map-reduce partition columns: p_partkey (type: int) - Statistics: Num rows: 325 Data size: 1300 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE tag: 1 auto parallelism: true Path -> Alias: @@ -1948,6 +1975,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -1966,6 +1995,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -1981,13 +2012,13 @@ STAGE PLANS: Map Operator Tree: TableScan alias: part - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE GatherStats: false Reduce Output Operator key expressions: p_mfgr (type: string), p_name (type: string) sort order: ++ Map-reduce partition columns: p_mfgr (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true @@ -2008,6 +2039,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -2026,6 +2059,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -2046,17 +2081,17 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} {VALUE._col1} {VALUE._col2} {VALUE._col3} {VALUE._col4} {VALUE._col5} {VALUE._col6} {VALUE._col7} 1 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 357 Data size: 1430 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 14 Data size: 8823 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 357 Data size: 1430 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 14 Data size: 8823 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 357 Data size: 1430 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 14 Data size: 8823 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -2076,22 +2111,22 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Filter Operator isSamplingPred: false predicate: _col0 is not null (type: boolean) - Statistics: Num rows: 2 Data size: 1298 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 2 Data size: 1298 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE tag: 0 value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) auto parallelism: true @@ -2223,17 +2258,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: p1 - Statistics: Num rows: 649 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: p_partkey is not null (type: boolean) - Statistics: Num rows: 325 Data size: 1300 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: p_partkey (type: int) sort order: + Map-reduce partition columns: p_partkey (type: int) - Statistics: Num rows: 325 Data size: 1300 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE tag: 0 auto parallelism: true Path -> Alias: @@ -2253,6 +2288,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -2271,6 +2308,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -2286,13 +2325,13 @@ STAGE PLANS: Map Operator Tree: TableScan alias: part - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE GatherStats: false Reduce Output Operator key expressions: p_mfgr (type: string), p_name (type: string) sort order: ++ Map-reduce partition columns: p_mfgr (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true @@ -2313,6 +2352,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -2331,6 +2372,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -2351,17 +2394,17 @@ STAGE PLANS: 0 1 {KEY.reducesinkkey0} {VALUE._col0} {VALUE._col1} {VALUE._col2} {VALUE._col3} {VALUE._col4} {VALUE._col5} {VALUE._col6} {VALUE._col7} outputColumnNames: _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20 - Statistics: Num rows: 357 Data size: 1430 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 14 Data size: 8823 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col12 (type: int), _col13 (type: string), _col14 (type: string), _col15 (type: string), _col16 (type: string), _col17 (type: int), _col18 (type: string), _col19 (type: double), _col20 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 357 Data size: 1430 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 14 Data size: 8823 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 357 Data size: 1430 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 14 Data size: 8823 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -2381,22 +2424,22 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Filter Operator isSamplingPred: false predicate: _col0 is not null (type: boolean) - Statistics: Num rows: 2 Data size: 1298 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 2 Data size: 1298 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE tag: 1 value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) auto parallelism: true @@ -2537,15 +2580,15 @@ STAGE PLANS: Map Operator Tree: TableScan alias: part - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE GatherStats: false PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: p_mfgr (type: string), p_name (type: string), p_size (type: int) sort order: ++- Map-reduce partition columns: p_mfgr (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true @@ -2566,6 +2609,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -2584,6 +2629,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -2598,18 +2645,18 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int) outputColumnNames: _col1, _col2, _col5 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col2 (type: string), _col1 (type: string), _col5 (type: int) sort order: ++- Map-reduce partition columns: _col2 (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int) auto parallelism: true @@ -2617,19 +2664,19 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col2 (type: string), _col1 (type: string), _col5 (type: int), _wcol0 (type: int) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -2810,15 +2857,15 @@ STAGE PLANS: Map Operator Tree: TableScan alias: part - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE GatherStats: false PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: p_mfgr (type: string), p_name (type: string) sort order: ++ Map-reduce partition columns: p_mfgr (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true @@ -2839,6 +2886,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -2857,6 +2906,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -2871,18 +2922,18 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int), _col7 (type: double) outputColumnNames: _col1, _col2, _col5, _col7 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col2 (type: string), _col1 (type: string) sort order: ++ Map-reduce partition columns: _col2 (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int), _col7 (type: double) auto parallelism: true @@ -2890,19 +2941,19 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col2 (type: string), _col1 (type: string), _col5 (type: int), _wcol0 (type: int), _wcol1 (type: int), _wcol2 (type: double) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -3087,13 +3138,13 @@ STAGE PLANS: Map Operator Tree: TableScan alias: part - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE GatherStats: false Reduce Output Operator key expressions: p_mfgr (type: string), p_name (type: string) sort order: ++ Map-reduce partition columns: p_mfgr (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true @@ -3114,6 +3165,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -3132,6 +3185,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -3146,18 +3201,18 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int), _col7 (type: double) outputColumnNames: _col1, _col2, _col5, _col7 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col2 (type: string), _col1 (type: string) sort order: ++ Map-reduce partition columns: _col2 (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int), _col7 (type: double) auto parallelism: true @@ -3165,19 +3220,19 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col2 (type: string), _col1 (type: string), _col5 (type: int), _wcol0 (type: int), _wcol1 (type: int), _wcol2 (type: double) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -3372,13 +3427,13 @@ STAGE PLANS: Map Operator Tree: TableScan alias: part - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE GatherStats: false Reduce Output Operator key expressions: p_mfgr (type: string), p_mfgr (type: string), p_name (type: string) sort order: +++ Map-reduce partition columns: p_mfgr (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true @@ -3399,6 +3454,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -3417,6 +3474,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -3431,16 +3490,16 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col2 (type: string), _col2 (type: string), _col1 (type: string) sort order: +++ Map-reduce partition columns: _col2 (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) auto parallelism: true @@ -3448,18 +3507,18 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int), _col7 (type: double) outputColumnNames: _col1, _col2, _col5, _col7 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col2 (type: string), _col1 (type: string) sort order: ++ Map-reduce partition columns: _col2 (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int), _col7 (type: double) auto parallelism: true @@ -3467,19 +3526,19 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col2 (type: string), _col1 (type: string), _col5 (type: int), _wcol0 (type: int), _wcol1 (type: int), _wcol2 (type: double) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -3694,13 +3753,13 @@ STAGE PLANS: Map Operator Tree: TableScan alias: part - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE GatherStats: false Reduce Output Operator key expressions: p_mfgr (type: string), p_name (type: string) sort order: ++ Map-reduce partition columns: p_mfgr (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true @@ -3721,6 +3780,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -3739,6 +3800,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -3753,18 +3816,18 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int), _col7 (type: double) outputColumnNames: _col1, _col2, _col5, _col7 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col2 (type: string), _col1 (type: string) sort order: ++ Map-reduce partition columns: _col2 (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int), _col7 (type: double) auto parallelism: true @@ -3772,19 +3835,19 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col2 (type: string), _col1 (type: string), _wcol0 (type: bigint), _wcol1 (type: double) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -4076,17 +4139,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: p1 - Statistics: Num rows: 649 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: p_partkey is not null (type: boolean) - Statistics: Num rows: 325 Data size: 1300 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: p_partkey (type: int) sort order: + Map-reduce partition columns: p_partkey (type: int) - Statistics: Num rows: 325 Data size: 1300 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE tag: 1 auto parallelism: true Path -> Alias: @@ -4106,6 +4169,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -4124,6 +4189,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -4139,13 +4206,13 @@ STAGE PLANS: Map Operator Tree: TableScan alias: part - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE GatherStats: false Reduce Output Operator key expressions: p_mfgr (type: string), p_name (type: string) sort order: ++ Map-reduce partition columns: p_mfgr (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true @@ -4166,6 +4233,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -4184,6 +4253,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -4204,12 +4275,12 @@ STAGE PLANS: 0 {VALUE._col0} {VALUE._col1} {VALUE._col4} {VALUE._col6} 1 outputColumnNames: _col1, _col2, _col5, _col7 - Statistics: Num rows: 357 Data size: 1430 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 14 Data size: 8823 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col2 (type: string), _col1 (type: string) sort order: ++ Map-reduce partition columns: _col2 (type: string) - Statistics: Num rows: 357 Data size: 1430 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 14 Data size: 8823 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int), _col7 (type: double) auto parallelism: true @@ -4217,19 +4288,19 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 357 Data size: 1430 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 14 Data size: 8823 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 357 Data size: 1430 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 14 Data size: 8823 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col2 (type: string), _col1 (type: string), _wcol0 (type: int), _wcol1 (type: int), _wcol2 (type: bigint), _col7 (type: double), _wcol3 (type: double), _col5 (type: int), (_col5 - _wcol4) (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 357 Data size: 1430 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 14 Data size: 8823 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 357 Data size: 1430 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 14 Data size: 8823 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -4249,22 +4320,22 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col5 (type: int), _col7 (type: double) outputColumnNames: _col0, _col1, _col2, _col5, _col7 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Filter Operator isSamplingPred: false predicate: _col0 is not null (type: boolean) - Statistics: Num rows: 2 Data size: 1298 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) sort order: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 2 Data size: 1298 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE tag: 0 value expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int), _col7 (type: double) auto parallelism: true @@ -4394,13 +4465,13 @@ STAGE PLANS: Map Operator Tree: TableScan alias: part - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE GatherStats: false Reduce Output Operator key expressions: p_mfgr (type: string), p_name (type: string) sort order: ++ Map-reduce partition columns: p_mfgr (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true @@ -4421,6 +4492,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -4439,6 +4512,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -4453,23 +4528,23 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col2 (type: string), _col1 (type: string), _col5 (type: int) outputColumnNames: _col2, _col1, _col5 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: _col2 (type: string), _col1 (type: string), _col5 (type: int) mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: int) sort order: +++ Map-reduce partition columns: _col0 (type: string), _col1 (type: string), _col2 (type: int) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 auto parallelism: true Reducer 3 @@ -4479,17 +4554,17 @@ STAGE PLANS: keys: KEY._col0 (type: string), KEY._col1 (type: string), KEY._col2 (type: int) mode: mergepartial outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 2 Data size: 1298 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col1 (type: string), _col2 (type: int) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 2 Data size: 1298 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 2 Data size: 1298 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -4662,23 +4737,23 @@ STAGE PLANS: Map Operator Tree: TableScan alias: part - Statistics: Num rows: 12 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE GatherStats: false Select Operator expressions: p_mfgr (type: string), p_brand (type: string), p_retailprice (type: double) outputColumnNames: p_mfgr, p_brand, p_retailprice - Statistics: Num rows: 12 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: sum(p_retailprice) keys: p_mfgr (type: string), p_brand (type: string) mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 12 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string), _col1 (type: string) sort order: ++ Map-reduce partition columns: _col0 (type: string), _col1 (type: string) - Statistics: Num rows: 12 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col2 (type: double) auto parallelism: true @@ -4699,6 +4774,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -4717,6 +4794,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -4736,16 +4815,16 @@ STAGE PLANS: keys: KEY._col0 (type: string), KEY._col1 (type: string) mode: mergepartial outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 6 Data size: 1298 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col1 (type: string), _col2 (type: double) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 6 Data size: 1298 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string), _col0 (type: string) sort order: ++ Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 6 Data size: 1298 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: double) auto parallelism: true @@ -4754,14 +4833,14 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 6 Data size: 1298 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 6 Data size: 1298 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string), _col1 (type: string) sort order: ++ Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 6 Data size: 1298 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: double) auto parallelism: true @@ -4769,19 +4848,19 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 6 Data size: 1298 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 6 Data size: 1298 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col1 (type: string), _col2 (type: double), _wcol0 (type: double) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 6 Data size: 1298 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 6 Data size: 1298 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -5133,13 +5212,13 @@ STAGE PLANS: Map Operator Tree: TableScan alias: part - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE GatherStats: false Reduce Output Operator key expressions: p_mfgr (type: string), p_name (type: string) sort order: ++ Map-reduce partition columns: p_mfgr (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true @@ -5160,6 +5239,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -5178,6 +5259,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -5192,26 +5275,26 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col2 (type: string), _col5 (type: int) sort order: ++ Map-reduce partition columns: _col2 (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int) auto parallelism: true Select Operator expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int), _col7 (type: double) outputColumnNames: _col1, _col2, _col5, _col7 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col2 (type: string), _col1 (type: string) sort order: ++ Map-reduce partition columns: _col2 (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int), _col7 (type: double) auto parallelism: true @@ -5219,14 +5302,14 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col2 (type: string), _col2 (type: string), _col1 (type: string) sort order: +++ Map-reduce partition columns: _col2 (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _wcol0 (type: bigint), _col1 (type: string), _col2 (type: string), _col5 (type: int) auto parallelism: true @@ -5234,19 +5317,19 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col3 (type: string), _col2 (type: string), _col6 (type: int), UDFToInteger(round(_col0, 1)) (type: int), _wcol1 (type: int), _wcol2 (type: int), _wcol3 (type: double), _wcol4 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 2 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -5271,19 +5354,19 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col2 (type: string), _col1 (type: string), _col5 (type: int), _wcol0 (type: int), _wcol1 (type: int), _wcol2 (type: double) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 1 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -5378,8 +5461,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@part PREHOOK: Output: default@part_4 PREHOOK: Output: default@part_5 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: from noop(on part partition by p_mfgr order by p_name) @@ -5639,13 +5720,13 @@ STAGE PLANS: Map Operator Tree: TableScan alias: part - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE GatherStats: false Reduce Output Operator key expressions: p_mfgr (type: string), p_mfgr (type: string) sort order: ++ Map-reduce partition columns: p_mfgr (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true @@ -5666,6 +5747,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -5684,6 +5767,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -5698,16 +5783,16 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col2 (type: string), _col1 (type: string), _col2 (type: string), _col1 (type: string) sort order: ++++ Map-reduce partition columns: _col2 (type: string), _col1 (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) auto parallelism: true @@ -5715,18 +5800,18 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int) outputColumnNames: _col1, _col2, _col5 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col2 (type: string), _col1 (type: string), _col2 (type: string), _col1 (type: string) sort order: ++++ Map-reduce partition columns: _col2 (type: string), _col1 (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int) auto parallelism: true @@ -5734,19 +5819,19 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col2 (type: string), _col1 (type: string), _wcol0 (type: int), _wcol1 (type: int), _col5 (type: int), _wcol2 (type: bigint) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -5992,13 +6077,13 @@ STAGE PLANS: Map Operator Tree: TableScan alias: part - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE GatherStats: false Reduce Output Operator key expressions: p_mfgr (type: string), p_mfgr (type: string) sort order: ++ Map-reduce partition columns: p_mfgr (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true @@ -6019,6 +6104,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -6037,6 +6124,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -6051,14 +6140,14 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col2 (type: string), _col1 (type: string), _col2 (type: string), _col1 (type: string) sort order: ++++ Map-reduce partition columns: _col2 (type: string), _col1 (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) auto parallelism: true @@ -6066,14 +6155,14 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col2 (type: string), _col2 (type: string) sort order: ++ Map-reduce partition columns: _col2 (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) auto parallelism: true @@ -6081,18 +6170,18 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int) outputColumnNames: _col1, _col2, _col5 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col2 (type: string), _col1 (type: string) sort order: ++ Map-reduce partition columns: _col2 (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int) auto parallelism: true @@ -6100,19 +6189,19 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col2 (type: string), _col1 (type: string), _wcol0 (type: int), _wcol1 (type: int), _col5 (type: int), _wcol2 (type: bigint) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -6341,13 +6430,13 @@ STAGE PLANS: Map Operator Tree: TableScan alias: part - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE GatherStats: false Reduce Output Operator key expressions: p_mfgr (type: string), p_name (type: string), p_mfgr (type: string), p_name (type: string) sort order: ++++ Map-reduce partition columns: p_mfgr (type: string), p_name (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true @@ -6368,6 +6457,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -6386,6 +6477,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -6400,14 +6493,14 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col2 (type: string), _col2 (type: string) sort order: ++ Map-reduce partition columns: _col2 (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) auto parallelism: true @@ -6415,18 +6508,18 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int) outputColumnNames: _col1, _col2, _col5 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col2 (type: string), _col1 (type: string) sort order: ++ Map-reduce partition columns: _col2 (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int) auto parallelism: true @@ -6434,19 +6527,19 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col2 (type: string), _col1 (type: string), _wcol0 (type: int), _wcol1 (type: int), _col5 (type: int), _wcol2 (type: bigint) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -6687,13 +6780,13 @@ STAGE PLANS: Map Operator Tree: TableScan alias: part - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE GatherStats: false Reduce Output Operator key expressions: p_mfgr (type: string), p_name (type: string), p_mfgr (type: string), p_name (type: string) sort order: ++++ Map-reduce partition columns: p_mfgr (type: string), p_name (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true @@ -6714,6 +6807,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -6732,6 +6827,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -6746,14 +6843,14 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col2 (type: string), _col2 (type: string) sort order: ++ Map-reduce partition columns: _col2 (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) auto parallelism: true @@ -6761,16 +6858,16 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col2 (type: string), _col1 (type: string), _col2 (type: string), _col1 (type: string) sort order: ++++ Map-reduce partition columns: _col2 (type: string), _col1 (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) auto parallelism: true @@ -6778,18 +6875,18 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int) outputColumnNames: _col1, _col2, _col5 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col2 (type: string), _col1 (type: string), _col2 (type: string), _col1 (type: string) sort order: ++++ Map-reduce partition columns: _col2 (type: string), _col1 (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int) auto parallelism: true @@ -6797,19 +6894,19 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col2 (type: string), _col1 (type: string), _wcol0 (type: int), _wcol1 (type: int), _col5 (type: int), _wcol2 (type: bigint) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -7081,13 +7178,13 @@ STAGE PLANS: Map Operator Tree: TableScan alias: part - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE GatherStats: false Reduce Output Operator key expressions: p_mfgr (type: string), p_name (type: string), p_mfgr (type: string), p_name (type: string) sort order: ++++ Map-reduce partition columns: p_mfgr (type: string), p_name (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true @@ -7108,6 +7205,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -7126,6 +7225,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -7140,16 +7241,16 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col2 (type: string), _col2 (type: string) sort order: ++ Map-reduce partition columns: _col2 (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) auto parallelism: true @@ -7157,18 +7258,18 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int) outputColumnNames: _col1, _col2, _col5 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col2 (type: string), _col1 (type: string), _col2 (type: string), _col1 (type: string) sort order: ++++ Map-reduce partition columns: _col2 (type: string), _col1 (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int) auto parallelism: true @@ -7176,19 +7277,19 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col2 (type: string), _col1 (type: string), _wcol0 (type: int), _wcol1 (type: int), _col5 (type: int), _wcol2 (type: bigint), _wcol2 (type: bigint) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -7426,13 +7527,13 @@ STAGE PLANS: Map Operator Tree: TableScan alias: part - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE GatherStats: false Reduce Output Operator key expressions: p_mfgr (type: string), p_name (type: string), p_mfgr (type: string), p_name (type: string) sort order: ++++ Map-reduce partition columns: p_mfgr (type: string), p_name (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true @@ -7453,6 +7554,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -7471,6 +7574,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.part numFiles 1 + numRows 26 + rawDataSize 16042 serialization.ddl struct part { i32 p_partkey, string p_name, string p_mfgr, string p_brand, string p_type, i32 p_size, string p_container, double p_retailprice, string p_comment} serialization.format 1 serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde @@ -7485,16 +7590,16 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col2 (type: string), _col1 (type: string), _col2 (type: string), _col1 (type: string) sort order: ++++ Map-reduce partition columns: _col2 (type: string), _col1 (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) auto parallelism: true @@ -7502,18 +7607,18 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int) outputColumnNames: _col1, _col2, _col5 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col2 (type: string), _col1 (type: string) sort order: ++ Map-reduce partition columns: _col2 (type: string) - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: string), _col2 (type: string), _col5 (type: int) auto parallelism: true @@ -7521,19 +7626,19 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Extract - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col2 (type: string), _col1 (type: string), _wcol0 (type: int), _wcol1 (type: int), _col5 (type: int), _wcol2 (type: bigint), _wcol2 (type: bigint) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 4 Data size: 2597 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat diff --git ql/src/test/results/clientpositive/spark/vectorized_rcfile_columnar.q.out ql/src/test/results/clientpositive/spark/vectorized_rcfile_columnar.q.out index 984d904..ee8959b 100644 --- ql/src/test/results/clientpositive/spark/vectorized_rcfile_columnar.q.out +++ ql/src/test/results/clientpositive/spark/vectorized_rcfile_columnar.q.out @@ -27,7 +27,6 @@ INSERT OVERWRITE TABLE columnTable SELECT src.key, src.value LIMIT 10 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@columntable -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM src INSERT OVERWRITE TABLE columnTable SELECT src.key, src.value LIMIT 10 POSTHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/spark/vectorized_timestamp_funcs.q.out ql/src/test/results/clientpositive/spark/vectorized_timestamp_funcs.q.out index cd43197..b01d9a3 100644 --- ql/src/test/results/clientpositive/spark/vectorized_timestamp_funcs.q.out +++ ql/src/test/results/clientpositive/spark/vectorized_timestamp_funcs.q.out @@ -25,7 +25,6 @@ LIMIT 40 PREHOOK: type: QUERY PREHOOK: Input: default@alltypesorc PREHOOK: Output: default@alltypesorc_string -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE alltypesorc_string SELECT to_utc_timestamp(ctimestamp1, 'America/Los_Angeles'), @@ -49,7 +48,6 @@ PREHOOK: query: INSERT INTO TABLE alltypesorc_wrong SELECT 'abcd' FROM alltypeso PREHOOK: type: QUERY PREHOOK: Input: default@alltypesorc PREHOOK: Output: default@alltypesorc_wrong -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT INTO TABLE alltypesorc_wrong SELECT 'abcd' FROM alltypesorc LIMIT 1 POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc @@ -59,7 +57,6 @@ PREHOOK: query: INSERT INTO TABLE alltypesorc_wrong SELECT '2000:01:01 00-00-00' PREHOOK: type: QUERY PREHOOK: Input: default@alltypesorc PREHOOK: Output: default@alltypesorc_wrong -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT INTO TABLE alltypesorc_wrong SELECT '2000:01:01 00-00-00' FROM alltypesorc LIMIT 1 POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc @@ -69,7 +66,6 @@ PREHOOK: query: INSERT INTO TABLE alltypesorc_wrong SELECT '0000-00-00 99:99:99' PREHOOK: type: QUERY PREHOOK: Input: default@alltypesorc PREHOOK: Output: default@alltypesorc_wrong -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT INTO TABLE alltypesorc_wrong SELECT '0000-00-00 99:99:99' FROM alltypesorc LIMIT 1 POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc @@ -116,15 +112,15 @@ STAGE PLANS: Map Operator Tree: TableScan alias: alltypesorc_string - Statistics: Num rows: 20 Data size: 829 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 5694 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: to_unix_timestamp(ctimestamp1) (type: bigint), year(ctimestamp1) (type: int), month(ctimestamp1) (type: int), day(ctimestamp1) (type: int), dayofmonth(ctimestamp1) (type: int), weekofyear(ctimestamp1) (type: int), hour(ctimestamp1) (type: int), minute(ctimestamp1) (type: int), second(ctimestamp1) (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 20 Data size: 829 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 5694 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: bigint) sort order: + - Statistics: Num rows: 20 Data size: 829 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 5694 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int), _col5 (type: int), _col6 (type: int), _col7 (type: int), _col8 (type: int) Execution mode: vectorized Reducer 2 @@ -132,10 +128,10 @@ STAGE PLANS: Select Operator expressions: KEY.reducesinkkey0 (type: bigint), VALUE._col0 (type: int), VALUE._col1 (type: int), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: int), VALUE._col5 (type: int), VALUE._col6 (type: int), VALUE._col7 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 20 Data size: 829 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 5694 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 20 Data size: 829 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 5694 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -259,15 +255,15 @@ STAGE PLANS: Map Operator Tree: TableScan alias: alltypesorc_string - Statistics: Num rows: 8 Data size: 829 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 5694 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: to_unix_timestamp(stimestamp1) (type: bigint), year(stimestamp1) (type: int), month(stimestamp1) (type: int), day(stimestamp1) (type: int), dayofmonth(stimestamp1) (type: int), weekofyear(stimestamp1) (type: int), hour(stimestamp1) (type: int), minute(stimestamp1) (type: int), second(stimestamp1) (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 8 Data size: 829 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 5694 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: bigint) sort order: + - Statistics: Num rows: 8 Data size: 829 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 5694 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int), _col5 (type: int), _col6 (type: int), _col7 (type: int), _col8 (type: int) Execution mode: vectorized Reducer 2 @@ -275,10 +271,10 @@ STAGE PLANS: Select Operator expressions: KEY.reducesinkkey0 (type: bigint), VALUE._col0 (type: int), VALUE._col1 (type: int), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: int), VALUE._col5 (type: int), VALUE._col6 (type: int), VALUE._col7 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 8 Data size: 829 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 5694 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 8 Data size: 829 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 5694 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -402,15 +398,15 @@ STAGE PLANS: Map Operator Tree: TableScan alias: alltypesorc_string - Statistics: Num rows: 5 Data size: 829 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 5694 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: (to_unix_timestamp(ctimestamp1) = to_unix_timestamp(stimestamp1)) (type: boolean), (year(ctimestamp1) = year(stimestamp1)) (type: boolean), (month(ctimestamp1) = month(stimestamp1)) (type: boolean), (day(ctimestamp1) = day(stimestamp1)) (type: boolean), (dayofmonth(ctimestamp1) = dayofmonth(stimestamp1)) (type: boolean), (weekofyear(ctimestamp1) = weekofyear(stimestamp1)) (type: boolean), (hour(ctimestamp1) = hour(stimestamp1)) (type: boolean), (minute(ctimestamp1) = minute(stimestamp1)) (type: boolean), (second(ctimestamp1) = second(stimestamp1)) (type: boolean) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 5 Data size: 829 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 5694 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: boolean) sort order: + - Statistics: Num rows: 5 Data size: 829 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 5694 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: boolean), _col2 (type: boolean), _col3 (type: boolean), _col4 (type: boolean), _col5 (type: boolean), _col6 (type: boolean), _col7 (type: boolean), _col8 (type: boolean) Execution mode: vectorized Reducer 2 @@ -418,10 +414,10 @@ STAGE PLANS: Select Operator expressions: KEY.reducesinkkey0 (type: boolean), VALUE._col0 (type: boolean), VALUE._col1 (type: boolean), VALUE._col2 (type: boolean), VALUE._col3 (type: boolean), VALUE._col4 (type: boolean), VALUE._col5 (type: boolean), VALUE._col6 (type: boolean), VALUE._col7 (type: boolean) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 5 Data size: 829 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 5694 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 5 Data size: 829 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 5694 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -549,15 +545,15 @@ STAGE PLANS: Map Operator Tree: TableScan alias: alltypesorc_wrong - Statistics: Num rows: 6 Data size: 678 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 294 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: to_unix_timestamp(stimestamp1) (type: bigint), year(stimestamp1) (type: int), month(stimestamp1) (type: int), day(stimestamp1) (type: int), dayofmonth(stimestamp1) (type: int), weekofyear(stimestamp1) (type: int), hour(stimestamp1) (type: int), minute(stimestamp1) (type: int), second(stimestamp1) (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 6 Data size: 678 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 294 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: bigint) sort order: + - Statistics: Num rows: 6 Data size: 678 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 294 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: int), _col5 (type: int), _col6 (type: int), _col7 (type: int), _col8 (type: int) Execution mode: vectorized Reducer 2 @@ -565,10 +561,10 @@ STAGE PLANS: Select Operator expressions: KEY.reducesinkkey0 (type: bigint), VALUE._col0 (type: int), VALUE._col1 (type: int), VALUE._col2 (type: int), VALUE._col3 (type: int), VALUE._col4 (type: int), VALUE._col5 (type: int), VALUE._col6 (type: int), VALUE._col7 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 6 Data size: 678 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 294 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 6 Data size: 678 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 294 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -643,11 +639,11 @@ STAGE PLANS: Map Operator Tree: TableScan alias: alltypesorc_string - Statistics: Num rows: 20 Data size: 829 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 5694 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: ctimestamp1 (type: timestamp) outputColumnNames: ctimestamp1 - Statistics: Num rows: 20 Data size: 829 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 5694 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: min(ctimestamp1), max(ctimestamp1), count(ctimestamp1), count() mode: hash @@ -728,11 +724,11 @@ STAGE PLANS: Map Operator Tree: TableScan alias: alltypesorc_string - Statistics: Num rows: 20 Data size: 829 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 5694 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: ctimestamp1 (type: timestamp) outputColumnNames: ctimestamp1 - Statistics: Num rows: 20 Data size: 829 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 5694 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: sum(ctimestamp1) mode: hash @@ -818,11 +814,11 @@ STAGE PLANS: Map Operator Tree: TableScan alias: alltypesorc_string - Statistics: Num rows: 20 Data size: 829 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 5694 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: ctimestamp1 (type: timestamp) outputColumnNames: ctimestamp1 - Statistics: Num rows: 20 Data size: 829 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 40 Data size: 5694 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: avg(ctimestamp1), variance(ctimestamp1), var_pop(ctimestamp1), var_samp(ctimestamp1), std(ctimestamp1), stddev(ctimestamp1), stddev_pop(ctimestamp1), stddev_samp(ctimestamp1) mode: hash diff --git ql/src/test/results/clientpositive/spark/windowing.q.out ql/src/test/results/clientpositive/spark/windowing.q.out index 424c444..31d0511 100644 --- ql/src/test/results/clientpositive/spark/windowing.q.out +++ ql/src/test/results/clientpositive/spark/windowing.q.out @@ -1351,9 +1351,6 @@ PREHOOK: Input: default@part PREHOOK: Output: default@part_1 PREHOOK: Output: default@part_2 PREHOOK: Output: default@part_3 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: from part INSERT OVERWRITE TABLE part_1 select p_mfgr, p_name, p_size,