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/annotate_stats_join.q.out ql/src/test/results/clientpositive/spark/annotate_stats_join.q.out index a56ee53..c0e1f7d 100644 --- ql/src/test/results/clientpositive/spark/annotate_stats_join.q.out +++ ql/src/test/results/clientpositive/spark/annotate_stats_join.q.out @@ -74,7 +74,6 @@ PREHOOK: query: analyze table emp compute statistics PREHOOK: type: QUERY PREHOOK: Input: default@emp PREHOOK: Output: default@emp -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: analyze table emp compute statistics POSTHOOK: type: QUERY POSTHOOK: Input: default@emp @@ -83,7 +82,6 @@ PREHOOK: query: analyze table dept compute statistics PREHOOK: type: QUERY PREHOOK: Input: default@dept PREHOOK: Output: default@dept -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: analyze table dept compute statistics POSTHOOK: type: QUERY POSTHOOK: Input: default@dept @@ -92,7 +90,6 @@ PREHOOK: query: analyze table loc compute statistics PREHOOK: type: QUERY PREHOOK: Input: default@loc PREHOOK: Output: default@loc -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: analyze table loc compute statistics POSTHOOK: type: QUERY POSTHOOK: Input: default@loc @@ -172,29 +169,29 @@ STAGE PLANS: Map Operator Tree: TableScan alias: d - Statistics: Num rows: 0 Data size: 68 Basic stats: PARTIAL Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 62 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: deptid is not null (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: deptid (type: int) sort order: + Map-reduce partition columns: deptid (type: int) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE value expressions: deptname (type: string) Map 3 Map Operator Tree: TableScan alias: e - Statistics: Num rows: 5 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 48 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: deptid is not null (type: boolean) - Statistics: Num rows: 5 Data size: 495 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: deptid (type: int) sort order: + Map-reduce partition columns: deptid (type: int) - Statistics: Num rows: 5 Data size: 495 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE value expressions: lastname (type: string), locid (type: int) Reducer 2 Reduce Operator Tree: @@ -205,14 +202,14 @@ STAGE PLANS: 0 {VALUE._col0} {KEY.reducesinkkey0} {VALUE._col1} 1 {KEY.reducesinkkey0} {VALUE._col0} outputColumnNames: _col0, _col1, _col2, _col6, _col7 - Statistics: Num rows: 5 Data size: 544 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 41 Data size: 7954 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: _col0 (type: string), _col1 (type: int), _col2 (type: int), _col6 (type: int), _col7 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 5 Data size: 544 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 41 Data size: 7954 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 5 Data size: 544 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 41 Data size: 7954 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -247,28 +244,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: dept - Statistics: Num rows: 0 Data size: 68 Basic stats: PARTIAL Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 62 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: (deptid is not null and deptname is not null) (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: deptid (type: int), deptname (type: string) sort order: ++ Map-reduce partition columns: deptid (type: int), deptname (type: string) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE Map 3 Map Operator Tree: TableScan alias: emp - Statistics: Num rows: 5 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 48 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: (deptid is not null and lastname is not null) (type: boolean) - Statistics: Num rows: 5 Data size: 495 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: deptid (type: int), lastname (type: string) sort order: ++ Map-reduce partition columns: deptid (type: int), lastname (type: string) - Statistics: Num rows: 5 Data size: 495 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE value expressions: locid (type: int) Reducer 2 Reduce Operator Tree: @@ -279,17 +276,17 @@ STAGE PLANS: 0 {KEY.reducesinkkey1} {KEY.reducesinkkey0} {VALUE._col0} 1 {KEY.reducesinkkey0} {KEY.reducesinkkey1} outputColumnNames: _col0, _col1, _col2, _col6, _col7 - Statistics: Num rows: 5 Data size: 544 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 1164 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: ((_col1 = _col6) and (_col0 = _col7)) (type: boolean) - Statistics: Num rows: 1 Data size: 108 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 194 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: _col0 (type: string), _col1 (type: int), _col2 (type: int), _col6 (type: int), _col7 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 1 Data size: 108 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 194 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 108 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 194 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -320,28 +317,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: d - Statistics: Num rows: 0 Data size: 68 Basic stats: PARTIAL Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 62 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: (deptid is not null and deptname is not null) (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: deptid (type: int), deptname (type: string) sort order: ++ Map-reduce partition columns: deptid (type: int), deptname (type: string) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE Map 3 Map Operator Tree: TableScan alias: e - Statistics: Num rows: 5 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 48 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: (deptid is not null and lastname is not null) (type: boolean) - Statistics: Num rows: 5 Data size: 495 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: deptid (type: int), lastname (type: string) sort order: ++ Map-reduce partition columns: deptid (type: int), lastname (type: string) - Statistics: Num rows: 5 Data size: 495 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE value expressions: locid (type: int) Reducer 2 Reduce Operator Tree: @@ -352,14 +349,14 @@ STAGE PLANS: 0 {KEY.reducesinkkey1} {KEY.reducesinkkey0} {VALUE._col0} 1 {KEY.reducesinkkey0} {KEY.reducesinkkey1} outputColumnNames: _col0, _col1, _col2, _col6, _col7 - Statistics: Num rows: 5 Data size: 544 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 1164 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: _col0 (type: string), _col1 (type: int), _col2 (type: int), _col6 (type: int), _col7 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Statistics: Num rows: 5 Data size: 544 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 1164 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 5 Data size: 544 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 1164 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -394,28 +391,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: dept - Statistics: Num rows: 0 Data size: 68 Basic stats: PARTIAL Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 62 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: (deptid is not null and deptname is not null) (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: deptid (type: int), deptname (type: string), deptname (type: string) sort order: +++ Map-reduce partition columns: deptid (type: int), deptname (type: string), deptname (type: string) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE Map 3 Map Operator Tree: TableScan alias: emp - Statistics: Num rows: 5 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 48 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: (deptid is not null and lastname is not null) (type: boolean) - Statistics: Num rows: 5 Data size: 495 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: deptid (type: int), lastname (type: string), lastname (type: string) sort order: +++ Map-reduce partition columns: deptid (type: int), lastname (type: string), lastname (type: string) - Statistics: Num rows: 5 Data size: 495 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE value expressions: locid (type: int) Reducer 2 Reduce Operator Tree: @@ -426,17 +423,17 @@ STAGE PLANS: 0 {KEY.reducesinkkey1} {KEY.reducesinkkey0} {VALUE._col0} 1 {KEY.reducesinkkey0} {KEY.reducesinkkey1} outputColumnNames: _col0, _col1, _col2, _col6, _col7 - Statistics: Num rows: 5 Data size: 544 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 2134 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: (((_col1 = _col6) and (_col0 = _col7)) and (_col7 = _col0)) (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 194 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: _col0 (type: string), _col1 (type: int), _col2 (type: int), _col6 (type: int), _col7 (type: string) 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: 194 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 194 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -471,43 +468,43 @@ STAGE PLANS: Map Operator Tree: TableScan alias: d - Statistics: Num rows: 0 Data size: 68 Basic stats: PARTIAL Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 62 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: deptid is not null (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: deptid (type: int) sort order: + Map-reduce partition columns: deptid (type: int) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE value expressions: deptname (type: string) Map 3 Map Operator Tree: TableScan alias: e1 - Statistics: Num rows: 5 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 48 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: deptid is not null (type: boolean) - Statistics: Num rows: 5 Data size: 495 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: deptid (type: int) sort order: + Map-reduce partition columns: deptid (type: int) - Statistics: Num rows: 5 Data size: 495 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE value expressions: lastname (type: string), locid (type: int) Map 4 Map Operator Tree: TableScan alias: e - Statistics: Num rows: 5 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 48 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: deptid is not null (type: boolean) - Statistics: Num rows: 5 Data size: 495 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: deptid (type: int) sort order: + Map-reduce partition columns: deptid (type: int) - Statistics: Num rows: 5 Data size: 495 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE value expressions: lastname (type: string), locid (type: int) Reducer 2 Reduce Operator Tree: @@ -520,14 +517,14 @@ STAGE PLANS: 1 {KEY.reducesinkkey0} {VALUE._col0} 2 {VALUE._col0} {KEY.reducesinkkey0} {VALUE._col1} outputColumnNames: _col0, _col1, _col2, _col6, _col7, _col11, _col12, _col13 - Statistics: Num rows: 11 Data size: 1089 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 658 Data size: 192794 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: _col0 (type: string), _col1 (type: int), _col2 (type: int), _col6 (type: int), _col7 (type: string), _col11 (type: string), _col12 (type: int), _col13 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 11 Data size: 1089 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 658 Data size: 192794 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 11 Data size: 1089 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 658 Data size: 192794 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -560,43 +557,43 @@ STAGE PLANS: Map Operator Tree: TableScan alias: d - Statistics: Num rows: 0 Data size: 68 Basic stats: PARTIAL Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 62 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: deptid is not null (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: deptid (type: int) sort order: + Map-reduce partition columns: deptid (type: int) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE value expressions: deptname (type: string) Map 3 Map Operator Tree: TableScan alias: e - Statistics: Num rows: 5 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 48 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: deptid is not null (type: boolean) - Statistics: Num rows: 5 Data size: 495 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: deptid (type: int) sort order: + Map-reduce partition columns: deptid (type: int) - Statistics: Num rows: 5 Data size: 495 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE value expressions: lastname (type: string), locid (type: int) Map 4 Map Operator Tree: TableScan alias: l - Statistics: Num rows: 1 Data size: 117 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 8 Data size: 109 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: locid is not null (type: boolean) - Statistics: Num rows: 1 Data size: 90 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: locid (type: int) sort order: + Map-reduce partition columns: locid (type: int) - Statistics: Num rows: 1 Data size: 90 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE value expressions: state (type: string), zip (type: bigint), year (type: int) Reducer 2 Reduce Operator Tree: @@ -609,14 +606,14 @@ STAGE PLANS: 1 {KEY.reducesinkkey0} {VALUE._col0} 2 {VALUE._col0} {KEY.reducesinkkey0} {VALUE._col1} {VALUE._col2} outputColumnNames: _col0, _col1, _col2, _col6, _col7, _col11, _col12, _col13, _col14 - Statistics: Num rows: 11 Data size: 1089 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 47 Data size: 13912 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: _col0 (type: string), _col1 (type: int), _col2 (type: int), _col6 (type: int), _col7 (type: string), _col11 (type: string), _col12 (type: int), _col13 (type: bigint), _col14 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 11 Data size: 1089 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 47 Data size: 13912 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 11 Data size: 1089 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 47 Data size: 13912 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -651,42 +648,42 @@ STAGE PLANS: Map Operator Tree: TableScan alias: d - Statistics: Num rows: 0 Data size: 68 Basic stats: PARTIAL Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 62 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: (deptid is not null and deptname is not null) (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: deptid (type: int), deptname (type: string) sort order: ++ Map-reduce partition columns: deptid (type: int), deptname (type: string) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: COMPLETE + Statistics: Num rows: 6 Data size: 570 Basic stats: COMPLETE Column stats: COMPLETE Map 3 Map Operator Tree: TableScan alias: e - Statistics: Num rows: 5 Data size: 600 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 48 Data size: 552 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: (deptid is not null and lastname is not null) (type: boolean) - Statistics: Num rows: 5 Data size: 495 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: deptid (type: int), lastname (type: string) sort order: ++ Map-reduce partition columns: deptid (type: int), lastname (type: string) - Statistics: Num rows: 5 Data size: 495 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 48 Data size: 4752 Basic stats: COMPLETE Column stats: COMPLETE value expressions: locid (type: int) Map 4 Map Operator Tree: TableScan alias: l - Statistics: Num rows: 1 Data size: 117 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 8 Data size: 109 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: (locid is not null and state is not null) (type: boolean) - Statistics: Num rows: 1 Data size: 90 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: locid (type: int), state (type: string) sort order: ++ Map-reduce partition columns: locid (type: int), state (type: string) - Statistics: Num rows: 1 Data size: 90 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE value expressions: zip (type: bigint), year (type: int) Reducer 2 Reduce Operator Tree: @@ -699,14 +696,14 @@ STAGE PLANS: 1 {KEY.reducesinkkey0} {KEY.reducesinkkey1} 2 {KEY.reducesinkkey1} {KEY.reducesinkkey0} {VALUE._col0} {VALUE._col1} outputColumnNames: _col0, _col1, _col2, _col6, _col7, _col11, _col12, _col13, _col14 - Statistics: Num rows: 11 Data size: 1089 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 296 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: _col0 (type: string), _col1 (type: int), _col2 (type: int), _col6 (type: int), _col7 (type: string), _col11 (type: string), _col12 (type: int), _col13 (type: bigint), _col14 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 11 Data size: 1089 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 296 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 11 Data size: 1089 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 296 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/auto_join1.q.out ql/src/test/results/clientpositive/spark/auto_join1.q.out index 222fe6b..9dff27e 100644 --- ql/src/test/results/clientpositive/spark/auto_join1.q.out +++ ql/src/test/results/clientpositive/spark/auto_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/auto_join14.q.out ql/src/test/results/clientpositive/spark/auto_join14.q.out index e1e6a4b..0e911de 100644 --- ql/src/test/results/clientpositive/spark/auto_join14.q.out +++ ql/src/test/results/clientpositive/spark/auto_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/auto_join17.q.out ql/src/test/results/clientpositive/spark/auto_join17.q.out index 8985dd4..0a852c7 100644 --- ql/src/test/results/clientpositive/spark/auto_join17.q.out +++ ql/src/test/results/clientpositive/spark/auto_join17.q.out @@ -99,7 +99,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/auto_join19.q.out ql/src/test/results/clientpositive/spark/auto_join19.q.out index 974aec5..fce4982 100644 --- ql/src/test/results/clientpositive/spark/auto_join19.q.out +++ ql/src/test/results/clientpositive/spark/auto_join19.q.out @@ -106,7 +106,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@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' or src1.ds = '2008-04-09' )and (src1.hr = '12' or src1.hr = '11') diff --git ql/src/test/results/clientpositive/spark/auto_join2.q.out ql/src/test/results/clientpositive/spark/auto_join2.q.out index 49544a1..463efde 100644 --- ql/src/test/results/clientpositive/spark/auto_join2.q.out +++ ql/src/test/results/clientpositive/spark/auto_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/auto_join24.q.out ql/src/test/results/clientpositive/spark/auto_join24.q.out index 15b8888..7d75479 100644 --- ql/src/test/results/clientpositive/spark/auto_join24.q.out +++ ql/src/test/results/clientpositive/spark/auto_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 @@ -41,28 +40,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 17 Data size: 1791 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 309 Data size: 1482 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 9 Data size: 948 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 155 Data size: 743 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: string) sort order: + Map-reduce partition columns: key (type: string) - Statistics: Num rows: 9 Data size: 948 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 155 Data size: 743 Basic stats: COMPLETE Column stats: NONE Map 4 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 17 Data size: 1791 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 309 Data size: 1482 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 9 Data size: 948 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 155 Data size: 743 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: string) sort order: + Map-reduce partition columns: key (type: string) - Statistics: Num rows: 9 Data size: 948 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 155 Data size: 743 Basic stats: COMPLETE Column stats: NONE value expressions: cnt (type: int) Reducer 2 Reduce Operator Tree: @@ -73,11 +72,11 @@ STAGE PLANS: 0 {VALUE._col0} 1 outputColumnNames: _col1 - Statistics: Num rows: 9 Data size: 1042 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 170 Data size: 817 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col1 (type: int) outputColumnNames: _col1 - Statistics: Num rows: 9 Data size: 1042 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 170 Data size: 817 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: sum(_col1) mode: hash diff --git ql/src/test/results/clientpositive/spark/auto_join25.q.out ql/src/test/results/clientpositive/spark/auto_join25.q.out index f0f45cb..bfa0486 100644 --- ql/src/test/results/clientpositive/spark/auto_join25.q.out +++ ql/src/test/results/clientpositive/spark/auto_join25.q.out @@ -20,7 +20,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@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' or src1.ds = '2008-04-09' )and (src1.hr = '12' or src1.hr = '11') @@ -62,7 +61,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 @@ -98,7 +96,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/auto_join3.q.out ql/src/test/results/clientpositive/spark/auto_join3.q.out index 64940de..3749908 100644 --- ql/src/test/results/clientpositive/spark/auto_join3.q.out +++ ql/src/test/results/clientpositive/spark/auto_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/auto_join4.q.out ql/src/test/results/clientpositive/spark/auto_join4.q.out index 014d3be..6faad30 100644 --- ql/src/test/results/clientpositive/spark/auto_join4.q.out +++ ql/src/test/results/clientpositive/spark/auto_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/auto_join5.q.out ql/src/test/results/clientpositive/spark/auto_join5.q.out index decc677..8455f02 100644 --- ql/src/test/results/clientpositive/spark/auto_join5.q.out +++ ql/src/test/results/clientpositive/spark/auto_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/auto_join6.q.out ql/src/test/results/clientpositive/spark/auto_join6.q.out index 8d7c324..b5e0687 100644 --- ql/src/test/results/clientpositive/spark/auto_join6.q.out +++ ql/src/test/results/clientpositive/spark/auto_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/auto_join7.q.out ql/src/test/results/clientpositive/spark/auto_join7.q.out index fcb435d..4d42841 100644 --- ql/src/test/results/clientpositive/spark/auto_join7.q.out +++ ql/src/test/results/clientpositive/spark/auto_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/auto_join8.q.out ql/src/test/results/clientpositive/spark/auto_join8.q.out index 0ef5e16..4a8b765 100644 --- ql/src/test/results/clientpositive/spark/auto_join8.q.out +++ ql/src/test/results/clientpositive/spark/auto_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/auto_join9.q.out ql/src/test/results/clientpositive/spark/auto_join9.q.out index 2c4f277..cf0d086 100644 --- ql/src/test/results/clientpositive/spark/auto_join9.q.out +++ ql/src/test/results/clientpositive/spark/auto_join9.q.out @@ -100,7 +100,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/auto_join_reordering_values.q.out ql/src/test/results/clientpositive/spark/auto_join_reordering_values.q.out index b9f4666..b98f8dc 100644 --- ql/src/test/results/clientpositive/spark/auto_join_reordering_values.q.out +++ ql/src/test/results/clientpositive/spark/auto_join_reordering_values.q.out @@ -34,7 +34,6 @@ PREHOOK: query: insert overwrite table orderpayment_small select 748, '2011-03-2 PREHOOK: type: QUERY PREHOOK: Input: default@testsrc PREHOOK: Output: default@orderpayment_small -[Error 30017]: Skipping 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 orderpayment_small select 748, '2011-03-24', '2011-03-24', 55 ,5372613 from testsrc tablesample (1 rows) POSTHOOK: type: QUERY POSTHOOK: Input: default@testsrc @@ -60,7 +59,6 @@ PREHOOK: query: insert overwrite table user_small select key from testsrc tables PREHOOK: type: QUERY PREHOOK: Input: default@testsrc PREHOOK: Output: default@user_small -[Error 30017]: Skipping 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 user_small select key from testsrc tablesample (100 rows) POSTHOOK: type: QUERY POSTHOOK: Input: default@testsrc @@ -187,17 +185,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: dim_pay_date - Statistics: Num rows: 0 Data size: 37 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: date is not null (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: date (type: string) sort order: + Map-reduce partition columns: date (type: string) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE tag: 1 auto parallelism: true Path -> Alias: @@ -217,8 +215,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.orderpayment_small numFiles 1 - numRows -1 - rawDataSize -1 + numRows 1 + rawDataSize 36 serialization.ddl struct orderpayment_small { i32 dealid, string date, string time, i32 cityid, i32 userid} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -237,8 +235,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.orderpayment_small numFiles 1 - numRows -1 - rawDataSize -1 + numRows 1 + rawDataSize 36 serialization.ddl struct orderpayment_small { i32 dealid, string date, string time, i32 cityid, i32 userid} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -253,17 +251,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: deal - Statistics: Num rows: 9 Data size: 37 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: dealid is not null (type: boolean) - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: dealid (type: int) sort order: + Map-reduce partition columns: dealid (type: int) - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE tag: 1 auto parallelism: true Path -> Alias: @@ -283,8 +281,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.orderpayment_small numFiles 1 - numRows -1 - rawDataSize -1 + numRows 1 + rawDataSize 36 serialization.ddl struct orderpayment_small { i32 dealid, string date, string time, i32 cityid, i32 userid} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -303,8 +301,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.orderpayment_small numFiles 1 - numRows -1 - rawDataSize -1 + numRows 1 + rawDataSize 36 serialization.ddl struct orderpayment_small { i32 dealid, string date, string time, i32 cityid, i32 userid} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -319,17 +317,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: order_city - Statistics: Num rows: 9 Data size: 37 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: cityid is not null (type: boolean) - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: cityid (type: int) sort order: + Map-reduce partition columns: cityid (type: int) - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE tag: 1 auto parallelism: true Path -> Alias: @@ -349,8 +347,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.orderpayment_small numFiles 1 - numRows -1 - rawDataSize -1 + numRows 1 + rawDataSize 36 serialization.ddl struct orderpayment_small { i32 dealid, string date, string time, i32 cityid, i32 userid} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -369,8 +367,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.orderpayment_small numFiles 1 - numRows -1 - rawDataSize -1 + numRows 1 + rawDataSize 36 serialization.ddl struct orderpayment_small { i32 dealid, string date, string time, i32 cityid, i32 userid} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -385,17 +383,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: orderpayment - Statistics: Num rows: 0 Data size: 37 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: (((date is not null and dealid is not null) and cityid is not null) and userid is not null) (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: date (type: string) sort order: + Map-reduce partition columns: date (type: string) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE tag: 0 value expressions: dealid (type: int), cityid (type: int), userid (type: int) auto parallelism: true @@ -416,8 +414,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.orderpayment_small numFiles 1 - numRows -1 - rawDataSize -1 + numRows 1 + rawDataSize 36 serialization.ddl struct orderpayment_small { i32 dealid, string date, string time, i32 cityid, i32 userid} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -436,8 +434,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.orderpayment_small numFiles 1 - numRows -1 - rawDataSize -1 + numRows 1 + rawDataSize 36 serialization.ddl struct orderpayment_small { i32 dealid, string date, string time, i32 cityid, i32 userid} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -452,17 +450,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: user - Statistics: Num rows: 97 Data size: 388 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 100 Data size: 288 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: userid is not null (type: boolean) - Statistics: Num rows: 49 Data size: 196 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 50 Data size: 144 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: userid (type: int) sort order: + Map-reduce partition columns: userid (type: int) - Statistics: Num rows: 49 Data size: 196 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 50 Data size: 144 Basic stats: COMPLETE Column stats: NONE tag: 1 auto parallelism: true Path -> Alias: @@ -482,8 +480,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.user_small numFiles 1 - numRows -1 - rawDataSize -1 + numRows 100 + rawDataSize 288 serialization.ddl struct user_small { i32 userid} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -502,8 +500,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.user_small numFiles 1 - numRows -1 - rawDataSize -1 + numRows 100 + rawDataSize 288 serialization.ddl struct user_small { i32 userid} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -524,12 +522,12 @@ STAGE PLANS: 0 {VALUE._col0} {VALUE._col2} {VALUE._col3} 1 {KEY.reducesinkkey0} outputColumnNames: _col0, _col3, _col4, _col9 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 39 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: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 39 Basic stats: COMPLETE Column stats: NONE tag: 0 value expressions: _col3 (type: int), _col4 (type: int), _col9 (type: string) auto parallelism: true @@ -543,12 +541,12 @@ STAGE PLANS: 0 {VALUE._col2} {VALUE._col3} {VALUE._col8} 1 {KEY.reducesinkkey0} outputColumnNames: _col3, _col4, _col9, _col16 - Statistics: Num rows: 5 Data size: 22 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 42 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col3 (type: int) sort order: + Map-reduce partition columns: _col3 (type: int) - Statistics: Num rows: 5 Data size: 22 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 42 Basic stats: COMPLETE Column stats: NONE tag: 0 value expressions: _col4 (type: int), _col9 (type: string), _col16 (type: int) auto parallelism: true @@ -562,12 +560,12 @@ STAGE PLANS: 0 {VALUE._col3} {VALUE._col8} {VALUE._col15} 1 outputColumnNames: _col4, _col9, _col16 - Statistics: Num rows: 5 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 46 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col4 (type: int) sort order: + Map-reduce partition columns: _col4 (type: int) - Statistics: Num rows: 5 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 46 Basic stats: COMPLETE Column stats: NONE tag: 0 value expressions: _col9 (type: string), _col16 (type: int) auto parallelism: true @@ -581,20 +579,20 @@ STAGE PLANS: 0 {VALUE._col8} {VALUE._col15} 1 outputColumnNames: _col9, _col16 - Statistics: Num rows: 53 Data size: 215 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 55 Data size: 158 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col9 (type: string), _col16 (type: int) outputColumnNames: _col0, _col1 - Statistics: Num rows: 53 Data size: 215 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 55 Data size: 158 Basic stats: COMPLETE Column stats: NONE Limit Number of rows: 5 - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 10 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 #### A masked pattern was here #### NumFilesPerFileSink: 1 - Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 10 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/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/auto_sortmerge_join_10.q.out ql/src/test/results/clientpositive/spark/auto_sortmerge_join_10.q.out index f608cc5..679ae59 100644 --- ql/src/test/results/clientpositive/spark/auto_sortmerge_join_10.q.out +++ ql/src/test/results/clientpositive/spark/auto_sortmerge_join_10.q.out @@ -19,7 +19,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 @@ -32,7 +31,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 @@ -115,19 +113,19 @@ 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 Reducer 3 Reduce Operator Tree: Join Operator @@ -136,9 +134,9 @@ STAGE PLANS: condition expressions: 0 1 - 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 @@ -237,44 +235,44 @@ 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: key - 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 Group By Operator aggregations: count() bucketGroup: true keys: key (type: int) mode: hash outputColumnNames: _col0, _col1 - 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 value expressions: _col1 (type: bigint) 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: Group By Operator @@ -282,16 +280,16 @@ STAGE PLANS: keys: KEY._col0 (type: int) mode: mergepartial outputColumnNames: _col0, _col1 - 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: _col0 (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 3 Reduce Operator Tree: Join Operator @@ -300,9 +298,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 diff --git ql/src/test/results/clientpositive/spark/auto_sortmerge_join_13.q.out ql/src/test/results/clientpositive/spark/auto_sortmerge_join_13.q.out index a5a281b..fe7f7a9 100644 --- ql/src/test/results/clientpositive/spark/auto_sortmerge_join_13.q.out +++ ql/src/test/results/clientpositive/spark/auto_sortmerge_join_13.q.out @@ -22,7 +22,6 @@ PREHOOK: query: insert overwrite table tbl1 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 POSTHOOK: Input: default@src @@ -33,7 +32,6 @@ PREHOOK: query: insert overwrite table tbl2 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 POSTHOOK: Input: default@src @@ -93,29 +91,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: @@ -126,18 +124,18 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 {KEY.reducesinkkey0} {VALUE._col0} outputColumnNames: _col0, _col1, _col5, _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), _col5 (type: int), _col6 (type: string) outputColumnNames: _col0, _col1, _col2, _col3 - 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), _col2 (type: int) 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 @@ -146,10 +144,10 @@ STAGE PLANS: Select Operator expressions: _col1 (type: string), _col3 (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 @@ -196,8 +194,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 key1, a.value value1, b.key key2, b.value value2 FROM tbl1 a JOIN tbl2 b @@ -310,29 +306,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: @@ -343,18 +339,18 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 {KEY.reducesinkkey0} {VALUE._col0} outputColumnNames: _col0, _col1, _col5, _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), _col5 (type: int), _col6 (type: string) outputColumnNames: _col0, _col1, _col2, _col3 - 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), _col2 (type: int) 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 @@ -363,10 +359,10 @@ STAGE PLANS: Select Operator expressions: _col1 (type: string), _col3 (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 @@ -413,8 +409,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 key1, a.value value1, b.key key2, b.value value2 FROM tbl1 a JOIN tbl2 b @@ -527,29 +521,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: @@ -560,18 +554,18 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 {KEY.reducesinkkey0} {VALUE._col0} outputColumnNames: _col0, _col1, _col5, _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), _col5 (type: int), _col6 (type: string) outputColumnNames: _col0, _col1, _col2, _col3 - 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), _col2 (type: int) 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 @@ -580,10 +574,10 @@ STAGE PLANS: Select Operator expressions: _col1 (type: string), _col3 (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 @@ -630,8 +624,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 key1, a.value value1, b.key key2, b.value value2 FROM tbl1 a JOIN tbl2 b diff --git ql/src/test/results/clientpositive/spark/auto_sortmerge_join_14.q.out ql/src/test/results/clientpositive/spark/auto_sortmerge_join_14.q.out index 2fc3bb6..c325539 100644 --- ql/src/test/results/clientpositive/spark/auto_sortmerge_join_14.q.out +++ ql/src/test/results/clientpositive/spark/auto_sortmerge_join_14.q.out @@ -18,7 +18,6 @@ PREHOOK: query: insert overwrite table tbl1 select * from src where key < 20 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 < 20 POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -29,7 +28,6 @@ PREHOOK: query: insert overwrite table tbl2 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 POSTHOOK: Input: default@src @@ -60,22 +58,22 @@ 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 Map 4 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 45 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: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - Statistics: Num rows: 45 Data size: 180 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator @@ -84,9 +82,9 @@ STAGE PLANS: condition expressions: 0 1 - Statistics: Num rows: 49 Data size: 198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 49 Data size: 198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash @@ -136,7 +134,6 @@ PREHOOK: query: insert overwrite table tbl2 select * from src where key < 200 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 < 200 POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -167,22 +164,22 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 520 Data size: 2080 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 189 Data size: 1891 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: 520 Data size: 2080 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 189 Data size: 1891 Basic stats: COMPLETE Column stats: NONE Map 4 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 45 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: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - Statistics: Num rows: 45 Data size: 180 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator @@ -191,9 +188,9 @@ STAGE PLANS: condition expressions: 0 1 - Statistics: Num rows: 572 Data size: 2288 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 207 Data size: 2080 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 572 Data size: 2288 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 207 Data size: 2080 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash diff --git ql/src/test/results/clientpositive/spark/auto_sortmerge_join_15.q.out ql/src/test/results/clientpositive/spark/auto_sortmerge_join_15.q.out index 74cbd7c..ed53363 100644 --- ql/src/test/results/clientpositive/spark/auto_sortmerge_join_15.q.out +++ ql/src/test/results/clientpositive/spark/auto_sortmerge_join_15.q.out @@ -18,7 +18,6 @@ PREHOOK: query: insert overwrite table tbl1 select * from src where key < 20 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 < 20 POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -29,7 +28,6 @@ PREHOOK: query: insert overwrite table tbl2 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 POSTHOOK: Input: default@src @@ -58,22 +56,22 @@ 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 Map 4 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 45 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: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - Statistics: Num rows: 45 Data size: 180 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator @@ -82,9 +80,9 @@ STAGE PLANS: condition expressions: 0 1 - Statistics: Num rows: 49 Data size: 198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 49 Data size: 198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash @@ -141,22 +139,22 @@ 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 Map 4 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 45 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: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - Statistics: Num rows: 45 Data size: 180 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator @@ -165,9 +163,9 @@ STAGE PLANS: condition expressions: 0 1 - Statistics: Num rows: 49 Data size: 198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 49 Data size: 198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash diff --git ql/src/test/results/clientpositive/spark/auto_sortmerge_join_6.q.out ql/src/test/results/clientpositive/spark/auto_sortmerge_join_6.q.out index bc4a163..22a11ab 100644 --- ql/src/test/results/clientpositive/spark/auto_sortmerge_join_6.q.out +++ ql/src/test/results/clientpositive/spark/auto_sortmerge_join_6.q.out @@ -34,7 +34,6 @@ PREHOOK: query: insert overwrite table tbl1 select * from src 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 POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -45,7 +44,6 @@ PREHOOK: query: insert overwrite table tbl2 select * from src 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 POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -56,7 +54,6 @@ PREHOOK: query: insert overwrite table tbl3 select * from src PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@tbl3 -[Error 30017]: Skipping 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 tbl3 select * from src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -67,7 +64,6 @@ PREHOOK: query: insert overwrite table tbl4 select * from src PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@tbl4 -[Error 30017]: Skipping 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 tbl4 select * from src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -109,15 +105,15 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - 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 Map 5 Map Operator Tree: TableScan @@ -135,15 +131,15 @@ 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 Filter Operator predicate: (key is not null and value is not null) (type: boolean) - Statistics: Num rows: 14 Data size: 1479 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 125 Data size: 1328 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: 14 Data size: 1479 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Reducer 2 Reduce Operator Tree: @@ -154,12 +150,12 @@ STAGE PLANS: 0 {VALUE._col0} 1 outputColumnNames: _col1 - Statistics: Num rows: 799 Data size: 3198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col1 (type: string) sort order: + Map-reduce partition columns: _col1 (type: string) - Statistics: Num rows: 799 Data size: 3198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE Reducer 3 Reduce Operator Tree: Join Operator @@ -168,9 +164,9 @@ STAGE PLANS: condition expressions: 0 1 - Statistics: Num rows: 878 Data size: 3517 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 878 Data size: 3517 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash @@ -258,28 +254,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - 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 Map 6 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 Filter Operator predicate: (key is not null and value is not null) (type: boolean) - Statistics: Num rows: 14 Data size: 1479 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 125 Data size: 1328 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: 14 Data size: 1479 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Reducer 2 Reduce Operator Tree: @@ -289,9 +285,9 @@ STAGE PLANS: condition expressions: 0 1 - Statistics: Num rows: 878 Data size: 3517 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 878 Data size: 3517 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash @@ -328,12 +324,12 @@ STAGE PLANS: 0 {VALUE._col0} 1 outputColumnNames: _col1 - Statistics: Num rows: 799 Data size: 3198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col1 (type: string) sort order: + Map-reduce partition columns: _col1 (type: string) - Statistics: Num rows: 799 Data size: 3198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE Stage: Stage-0 Fetch Operator @@ -381,28 +377,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - 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 Map 5 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 Filter Operator predicate: (key is not null and value is not null) (type: boolean) - Statistics: Num rows: 14 Data size: 1479 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 125 Data size: 1328 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: 14 Data size: 1479 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Map 6 Map Operator Tree: @@ -426,12 +422,12 @@ STAGE PLANS: 0 {VALUE._col0} 1 outputColumnNames: _col1 - Statistics: Num rows: 799 Data size: 3198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col1 (type: string) sort order: + Map-reduce partition columns: _col1 (type: string) - Statistics: Num rows: 799 Data size: 3198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE Reducer 3 Reduce Operator Tree: Join Operator @@ -440,9 +436,9 @@ STAGE PLANS: condition expressions: 0 1 - Statistics: Num rows: 878 Data size: 3517 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 878 Data size: 3517 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash @@ -512,15 +508,15 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - 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: UDFToDouble(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: UDFToDouble(key) (type: double) sort order: + Map-reduce partition columns: UDFToDouble(key) (type: double) - 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 Map 4 Map Operator Tree: TableScan @@ -538,15 +534,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: UDFToDouble(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: UDFToDouble(key) (type: double) sort order: + Map-reduce partition columns: UDFToDouble(key) (type: double) - 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 @@ -557,9 +553,9 @@ STAGE PLANS: 0 1 2 - Statistics: Num rows: 1599 Data size: 6397 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 1599 Data size: 6397 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash @@ -629,41 +625,41 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - 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 Map 4 Map Operator Tree: TableScan alias: c - 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 Map 5 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 @@ -674,9 +670,9 @@ STAGE PLANS: 0 1 2 - Statistics: Num rows: 1599 Data size: 6397 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 1599 Data size: 6397 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash @@ -747,41 +743,41 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - 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 Map 5 Map Operator Tree: TableScan alias: c - Statistics: Num rows: 58 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: value is not null (type: boolean) - Statistics: Num rows: 29 Data size: 2906 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: value (type: string) sort order: + Map-reduce partition columns: value (type: string) - Statistics: Num rows: 29 Data size: 2906 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Map 6 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 Filter Operator predicate: (key is not null and value is not null) (type: boolean) - Statistics: Num rows: 14 Data size: 1479 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 125 Data size: 1328 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: 14 Data size: 1479 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Reducer 2 Reduce Operator Tree: @@ -792,12 +788,12 @@ STAGE PLANS: 0 {VALUE._col0} 1 outputColumnNames: _col1 - Statistics: Num rows: 799 Data size: 3198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col1 (type: string) sort order: + Map-reduce partition columns: _col1 (type: string) - Statistics: Num rows: 799 Data size: 3198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE Reducer 3 Reduce Operator Tree: Join Operator @@ -806,9 +802,9 @@ STAGE PLANS: condition expressions: 0 1 - Statistics: Num rows: 878 Data size: 3517 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 878 Data size: 3517 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash @@ -879,15 +875,15 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - 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 Map 5 Map Operator Tree: TableScan @@ -905,15 +901,15 @@ 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 Filter Operator predicate: (key is not null and value is not null) (type: boolean) - Statistics: Num rows: 14 Data size: 1479 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 125 Data size: 1328 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: 14 Data size: 1479 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Reducer 2 Reduce Operator Tree: @@ -924,12 +920,12 @@ STAGE PLANS: 0 {VALUE._col0} 1 outputColumnNames: _col1 - Statistics: Num rows: 799 Data size: 3198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col1 (type: string) sort order: + Map-reduce partition columns: _col1 (type: string) - Statistics: Num rows: 799 Data size: 3198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE Reducer 3 Reduce Operator Tree: Join Operator @@ -938,9 +934,9 @@ STAGE PLANS: condition expressions: 0 1 - Statistics: Num rows: 878 Data size: 3517 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 878 Data size: 3517 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash @@ -1010,15 +1006,15 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - 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: UDFToDouble(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: UDFToDouble(key) (type: double) sort order: + Map-reduce partition columns: UDFToDouble(key) (type: double) - 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 Map 4 Map Operator Tree: TableScan @@ -1036,15 +1032,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: UDFToDouble(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: UDFToDouble(key) (type: double) sort order: + Map-reduce partition columns: UDFToDouble(key) (type: double) - 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 @@ -1055,9 +1051,9 @@ STAGE PLANS: 0 1 2 - Statistics: Num rows: 1599 Data size: 6397 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 1599 Data size: 6397 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash @@ -1127,41 +1123,41 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - 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 Map 4 Map Operator Tree: TableScan alias: c - 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 Map 5 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 @@ -1172,9 +1168,9 @@ STAGE PLANS: 0 1 2 - Statistics: Num rows: 1599 Data size: 6397 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 1599 Data size: 6397 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash @@ -1245,41 +1241,41 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - 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 Map 5 Map Operator Tree: TableScan alias: c - Statistics: Num rows: 58 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: value is not null (type: boolean) - Statistics: Num rows: 29 Data size: 2906 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: value (type: string) sort order: + Map-reduce partition columns: value (type: string) - Statistics: Num rows: 29 Data size: 2906 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Map 6 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 Filter Operator predicate: (key is not null and value is not null) (type: boolean) - Statistics: Num rows: 14 Data size: 1479 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 125 Data size: 1328 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: 14 Data size: 1479 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Reducer 2 Reduce Operator Tree: @@ -1290,12 +1286,12 @@ STAGE PLANS: 0 {VALUE._col0} 1 outputColumnNames: _col1 - Statistics: Num rows: 799 Data size: 3198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col1 (type: string) sort order: + Map-reduce partition columns: _col1 (type: string) - Statistics: Num rows: 799 Data size: 3198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE Reducer 3 Reduce Operator Tree: Join Operator @@ -1304,9 +1300,9 @@ STAGE PLANS: condition expressions: 0 1 - Statistics: Num rows: 878 Data size: 3517 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 878 Data size: 3517 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash diff --git ql/src/test/results/clientpositive/spark/auto_sortmerge_join_9.q.out ql/src/test/results/clientpositive/spark/auto_sortmerge_join_9.q.out index a7f994f..437e955 100644 --- ql/src/test/results/clientpositive/spark/auto_sortmerge_join_9.q.out +++ ql/src/test/results/clientpositive/spark/auto_sortmerge_join_9.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 @@ -186,28 +184,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 @@ -217,22 +215,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: @@ -241,14 +239,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), _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 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 @@ -323,28 +321,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 @@ -354,22 +352,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: @@ -378,9 +376,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 @@ -497,54 +495,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 @@ -554,22 +552,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: @@ -578,16 +576,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: @@ -598,14 +596,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 @@ -619,22 +617,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: @@ -643,16 +641,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 @@ -739,36 +737,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 @@ -777,9 +775,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 @@ -877,32 +875,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 @@ -911,9 +909,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 @@ -1035,36 +1033,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 @@ -1073,9 +1071,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 @@ -1187,36 +1185,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 @@ -1225,9 +1223,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 @@ -1315,36 +1313,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 @@ -1353,9 +1351,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 @@ -1439,32 +1437,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 @@ -1473,9 +1471,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 @@ -1557,32 +1555,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 @@ -1591,9 +1589,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 @@ -1685,53 +1683,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 @@ -1742,9 +1740,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 @@ -1850,32 +1848,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 @@ -1884,9 +1882,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 @@ -1980,28 +1978,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 @@ -2010,9 +2008,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 @@ -2094,28 +2092,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 @@ -2125,22 +2123,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: @@ -2149,14 +2147,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), _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 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 @@ -2231,28 +2229,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 @@ -2262,22 +2260,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: @@ -2286,9 +2284,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 @@ -2405,54 +2403,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 @@ -2462,22 +2460,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: @@ -2486,16 +2484,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: @@ -2506,14 +2504,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 @@ -2527,22 +2525,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: @@ -2551,16 +2549,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 @@ -2647,36 +2645,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 @@ -2685,9 +2683,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 @@ -2785,32 +2783,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 @@ -2819,9 +2817,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 @@ -2943,36 +2941,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 @@ -2981,9 +2979,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 @@ -3095,36 +3093,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 @@ -3133,9 +3131,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 @@ -3219,32 +3217,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 @@ -3253,9 +3251,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 @@ -3337,32 +3335,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 @@ -3371,9 +3369,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 @@ -3465,53 +3463,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 @@ -3522,9 +3520,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 @@ -3630,32 +3628,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 @@ -3664,9 +3662,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 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/bucket_map_join_tez1.q.out ql/src/test/results/clientpositive/spark/bucket_map_join_tez1.q.out index a2a7fe1..320c935 100644 --- ql/src/test/results/clientpositive/spark/bucket_map_join_tez1.q.out +++ ql/src/test/results/clientpositive/spark/bucket_map_join_tez1.q.out @@ -78,7 +78,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@srcbucket_mapjoin_part PREHOOK: Input: default@srcbucket_mapjoin_part@ds=2008-04-08 PREHOOK: Output: default@tab_part@ds=2008-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 tab_part partition (ds='2008-04-08') select key,value from srcbucket_mapjoin_part POSTHOOK: type: QUERY @@ -101,7 +100,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@srcbucket_mapjoin PREHOOK: Input: default@srcbucket_mapjoin@ds=2008-04-08 PREHOOK: Output: default@tab@ds=2008-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 tab partition (ds='2008-04-08') select key,value from srcbucket_mapjoin POSTHOOK: type: QUERY @@ -133,29 +131,29 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - 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: 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 value expressions: value (type: string) Map 3 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 27 Data size: 2808 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 14 Data size: 1456 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 121 Data size: 1283 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: 14 Data size: 1456 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 121 Data size: 1283 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Reducer 2 Reduce Operator Tree: @@ -166,14 +164,14 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 {VALUE._col0} outputColumnNames: _col0, _col1, _col7 - 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), _col7 (type: string) outputColumnNames: _col0, _col1, _col2 - 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 File Output Operator compressed: false - 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 table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -215,15 +213,15 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 27 Data size: 2808 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 14 Data size: 1456 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 121 Data size: 1283 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: 14 Data size: 1456 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 121 Data size: 1283 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Map 3 Map Operator Tree: @@ -258,14 +256,14 @@ STAGE PLANS: 0 {VALUE._col0} {KEY.reducesinkkey0} 1 {VALUE._col0} outputColumnNames: _col0, _col1, _col3 - Statistics: Num rows: 15 Data size: 1601 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 133 Data size: 1411 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col1 (type: int), _col0 (type: double), _col3 (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 15 Data size: 1601 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 133 Data size: 1411 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 15 Data size: 1601 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 133 Data size: 1411 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -322,42 +320,42 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 27 Data size: 2808 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 14 Data size: 1456 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 121 Data size: 1283 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: 14 Data size: 1456 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 121 Data size: 1283 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Map 3 Map Operator Tree: TableScan alias: tab_part - 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 Map 6 Map Operator Tree: TableScan alias: tab - Statistics: Num rows: 27 Data size: 2808 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 14 Data size: 1456 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 121 Data size: 1283 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: 14 Data size: 1456 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 121 Data size: 1283 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Reducer 2 Reduce Operator Tree: @@ -368,14 +366,14 @@ STAGE PLANS: 0 {VALUE._col0} {KEY.reducesinkkey0} 1 {VALUE._col0} outputColumnNames: _col0, _col1, _col3 - Statistics: Num rows: 438 Data size: 1755 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 150 Data size: 1600 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col1 (type: int), _col0 (type: double), _col3 (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 438 Data size: 1755 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 150 Data size: 1600 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 438 Data size: 1755 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 150 Data size: 1600 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -389,22 +387,22 @@ STAGE PLANS: 0 1 {KEY.reducesinkkey0} {VALUE._col0} outputColumnNames: _col6, _col7 - Statistics: Num rows: 799 Data size: 3198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col6 (type: int), _col7 (type: string) outputColumnNames: _col6, _col7 - Statistics: Num rows: 799 Data size: 3198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: sum(substr(_col7, 5)) keys: _col6 (type: int) mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 799 Data size: 3198 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: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 799 Data size: 3198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: double) Reducer 5 Reduce Operator Tree: @@ -413,16 +411,16 @@ STAGE PLANS: keys: KEY._col0 (type: int) mode: mergepartial outputColumnNames: _col0, _col1 - Statistics: Num rows: 399 Data size: 1596 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 137 Data size: 1455 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col1 (type: double), _col0 (type: int) outputColumnNames: _col0, _col1 - Statistics: Num rows: 399 Data size: 1596 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 137 Data size: 1455 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col1 (type: int) sort order: + Map-reduce partition columns: _col1 (type: int) - Statistics: Num rows: 399 Data size: 1596 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 137 Data size: 1455 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: double) Stage: Stage-0 @@ -458,43 +456,43 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - 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: 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 value expressions: value (type: string) Map 3 Map Operator Tree: TableScan alias: x - Statistics: Num rows: 27 Data size: 2808 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 14 Data size: 1456 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 121 Data size: 1283 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: 14 Data size: 1456 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 121 Data size: 1283 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Map 6 Map Operator Tree: TableScan alias: y - Statistics: Num rows: 702 Data size: 2808 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 351 Data size: 1404 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 121 Data size: 1283 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: 351 Data size: 1404 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 121 Data size: 1283 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator @@ -504,14 +502,14 @@ STAGE PLANS: 0 {VALUE._col0} {KEY.reducesinkkey0} 1 {VALUE._col0} outputColumnNames: _col0, _col1, _col3 - Statistics: Num rows: 212 Data size: 849 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col1 (type: int), _col0 (type: double), _col3 (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 212 Data size: 849 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 212 Data size: 849 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -525,22 +523,22 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 outputColumnNames: _col0, _col1 - Statistics: Num rows: 386 Data size: 1544 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 133 Data size: 1411 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 386 Data size: 1544 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 133 Data size: 1411 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: sum(substr(_col1, 5)) keys: _col0 (type: int) mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 386 Data size: 1544 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 133 Data size: 1411 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: 386 Data size: 1544 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 133 Data size: 1411 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: double) Reducer 5 Reduce Operator Tree: @@ -549,16 +547,16 @@ STAGE PLANS: keys: KEY._col0 (type: int) mode: mergepartial outputColumnNames: _col0, _col1 - Statistics: Num rows: 193 Data size: 772 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 66 Data size: 700 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col1 (type: double), _col0 (type: int) outputColumnNames: _col0, _col1 - Statistics: Num rows: 193 Data size: 772 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 66 Data size: 700 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col1 (type: int) sort order: + Map-reduce partition columns: _col1 (type: int) - Statistics: Num rows: 193 Data size: 772 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 66 Data size: 700 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: double) Stage: Stage-0 @@ -592,42 +590,42 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 27 Data size: 2808 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 14 Data size: 1456 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 121 Data size: 1283 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: 14 Data size: 1456 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 121 Data size: 1283 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Map 3 Map Operator Tree: TableScan alias: c - Statistics: Num rows: 702 Data size: 2808 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 351 Data size: 1404 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 121 Data size: 1283 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: 351 Data size: 1404 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 121 Data size: 1283 Basic stats: COMPLETE Column stats: NONE Map 4 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 Filter Operator 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 value expressions: value (type: string) Reducer 2 Reduce Operator Tree: @@ -640,14 +638,14 @@ STAGE PLANS: 1 {VALUE._col0} 2 outputColumnNames: _col0, _col1, _col7 - Statistics: Num rows: 772 Data size: 3088 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col7 (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 772 Data size: 3088 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 772 Data size: 3088 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -683,43 +681,43 @@ STAGE PLANS: Map Operator Tree: TableScan alias: c - Statistics: Num rows: 27 Data size: 2808 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 14 Data size: 1456 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 121 Data size: 1283 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: 14 Data size: 1456 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 121 Data size: 1283 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Map 3 Map Operator Tree: TableScan alias: x - 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: 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 value expressions: value (type: string) Map 5 Map Operator Tree: TableScan alias: y - Statistics: Num rows: 702 Data size: 2808 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 351 Data size: 1404 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 121 Data size: 1283 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: 351 Data size: 1404 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 121 Data size: 1283 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator @@ -729,14 +727,14 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 {VALUE._col0} outputColumnNames: _col0, _col1, _col3 - Statistics: Num rows: 424 Data size: 1698 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 424 Data size: 1698 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 424 Data size: 1698 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -750,16 +748,16 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 outputColumnNames: _col0, _col1 - Statistics: Num rows: 386 Data size: 1544 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) outputColumnNames: _col0, _col1 - Statistics: Num rows: 386 Data size: 1544 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: + Map-reduce partition columns: _col0 (type: int) - Statistics: Num rows: 386 Data size: 1544 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string) Stage: Stage-0 @@ -796,15 +794,15 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - 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: 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 value expressions: value (type: string) Map 3 Map Operator Tree: @@ -839,14 +837,14 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 {VALUE._col0} outputColumnNames: _col0, _col1, _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 Select Operator expressions: _col0 (type: int), _col1 (type: double), _col3 (type: string) outputColumnNames: _col0, _col1, _col2 - 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 File Output Operator compressed: false - 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 table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -902,15 +900,15 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - 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: 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 value expressions: value (type: string) Map 3 Map Operator Tree: @@ -939,14 +937,14 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 {VALUE._col0} outputColumnNames: _col0, _col1, _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 Select Operator expressions: _col0 (type: int), _col1 (type: double), _col3 (type: string) outputColumnNames: _col0, _col1, _col2 - 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 File Output Operator compressed: false - 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 table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -1001,28 +999,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 58 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: value is not null (type: boolean) - Statistics: Num rows: 29 Data size: 2906 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: value (type: string) sort order: + Map-reduce partition columns: value (type: string) - Statistics: Num rows: 29 Data size: 2906 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Map 3 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 27 Data size: 2808 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: value is not null (type: boolean) - Statistics: Num rows: 14 Data size: 1456 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 121 Data size: 1283 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: value (type: string) sort order: + Map-reduce partition columns: value (type: string) - Statistics: Num rows: 14 Data size: 1456 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 121 Data size: 1283 Basic stats: COMPLETE Column stats: NONE value expressions: key (type: int) Reducer 2 Reduce Operator Tree: @@ -1033,14 +1031,14 @@ STAGE PLANS: 0 {VALUE._col0} {KEY.reducesinkkey0} 1 {KEY.reducesinkkey0} outputColumnNames: _col0, _col1, _col7 - Statistics: Num rows: 31 Data size: 3196 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), _col7 (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 31 Data size: 3196 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 31 Data size: 3196 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -1066,7 +1064,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@srcbucket_mapjoin PREHOOK: Input: default@srcbucket_mapjoin@ds=2008-04-08 PREHOOK: Output: default@tab1 -[Error 30017]: Skipping 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 tab1 select key,value from srcbucket_mapjoin POSTHOOK: type: QUERY @@ -1098,29 +1095,29 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - 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: 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 value expressions: value (type: string) Map 3 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 27 Data size: 2808 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 14 Data size: 1456 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 121 Data size: 1283 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: 14 Data size: 1456 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 121 Data size: 1283 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Reducer 2 Reduce Operator Tree: @@ -1131,14 +1128,14 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 {VALUE._col0} outputColumnNames: _col0, _col1, _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), _col6 (type: string) outputColumnNames: _col0, _col1, _col2 - 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 File Output Operator compressed: false - 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 table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -1170,42 +1167,42 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - 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: value 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: value (type: string) sort order: + Map-reduce partition columns: value (type: string) - 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 value expressions: key (type: int) Map 3 Map Operator Tree: TableScan alias: c - 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 Map 5 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 Filter Operator predicate: (key is not null and value is not null) (type: boolean) - Statistics: Num rows: 14 Data size: 1479 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 125 Data size: 1328 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: 14 Data size: 1479 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Reducer 2 Reduce Operator Tree: @@ -1216,14 +1213,14 @@ STAGE PLANS: 0 {VALUE._col0} 1 {VALUE._col0} outputColumnNames: _col0, _col12 - Statistics: Num rows: 878 Data size: 3517 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col12 (type: int) outputColumnNames: _col0, _col1 - Statistics: Num rows: 878 Data size: 3517 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 878 Data size: 3517 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -1237,12 +1234,12 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 outputColumnNames: _col0, _col1 - Statistics: Num rows: 799 Data size: 3198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col1 (type: string) sort order: + Map-reduce partition columns: _col1 (type: string) - Statistics: Num rows: 799 Data size: 3198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int) Stage: Stage-0 diff --git ql/src/test/results/clientpositive/spark/bucket_map_join_tez2.q.out ql/src/test/results/clientpositive/spark/bucket_map_join_tez2.q.out index 60e2a4c..0a49598 100644 --- ql/src/test/results/clientpositive/spark/bucket_map_join_tez2.q.out +++ ql/src/test/results/clientpositive/spark/bucket_map_join_tez2.q.out @@ -78,7 +78,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@srcbucket_mapjoin_part PREHOOK: Input: default@srcbucket_mapjoin_part@ds=2008-04-08 PREHOOK: Output: default@tab_part@ds=2008-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 tab_part partition (ds='2008-04-08') select key,value from srcbucket_mapjoin_part POSTHOOK: type: QUERY @@ -101,7 +100,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@srcbucket_mapjoin PREHOOK: Input: default@srcbucket_mapjoin@ds=2008-04-08 PREHOOK: Output: default@tab@ds=2008-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 tab partition (ds='2008-04-08') select key,value from srcbucket_mapjoin POSTHOOK: type: QUERY @@ -130,42 +128,42 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - 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: value 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: value (type: string) sort order: + Map-reduce partition columns: value (type: string) - 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 value expressions: key (type: int) Map 3 Map Operator Tree: TableScan alias: c - 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 Map 5 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 Filter Operator predicate: (key is not null and value is not null) (type: boolean) - Statistics: Num rows: 14 Data size: 1479 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 125 Data size: 1328 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: 14 Data size: 1479 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Reducer 2 Reduce Operator Tree: @@ -176,14 +174,14 @@ STAGE PLANS: 0 {VALUE._col0} 1 {VALUE._col0} outputColumnNames: _col0, _col12 - Statistics: Num rows: 878 Data size: 3517 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col12 (type: int) outputColumnNames: _col0, _col1 - Statistics: Num rows: 878 Data size: 3517 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 878 Data size: 3517 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -197,12 +195,12 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 outputColumnNames: _col0, _col1 - Statistics: Num rows: 799 Data size: 3198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col1 (type: string) sort order: + Map-reduce partition columns: _col1 (type: string) - Statistics: Num rows: 799 Data size: 3198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int) Stage: Stage-0 @@ -225,7 +223,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@srcbucket_mapjoin PREHOOK: Input: default@srcbucket_mapjoin@ds=2008-04-08 PREHOOK: Output: default@tab1 -[Error 30017]: Skipping 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 tab1 select key,value from srcbucket_mapjoin POSTHOOK: type: QUERY @@ -271,15 +268,15 @@ STAGE PLANS: Map Operator Tree: TableScan alias: a - Statistics: Num rows: 27 Data size: 2808 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: UDFToDouble(key) is not null (type: boolean) - Statistics: Num rows: 14 Data size: 1456 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 121 Data size: 1283 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: UDFToDouble(key) (type: double) sort order: + Map-reduce partition columns: UDFToDouble(key) (type: double) - Statistics: Num rows: 14 Data size: 1456 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 121 Data size: 1283 Basic stats: COMPLETE Column stats: NONE value expressions: key (type: int), value (type: string) Reducer 2 Reduce Operator Tree: @@ -330,36 +327,36 @@ STAGE PLANS: Map Operator Tree: TableScan alias: tab_part - 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 > 1) and key is not null) (type: boolean) - Statistics: Num rows: 242 Data size: 968 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int) outputColumnNames: _col0 - Statistics: Num rows: 242 Data size: 968 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 83 Data size: 881 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: 242 Data size: 968 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE Map 3 Map Operator Tree: TableScan alias: tab_part - 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 > 2) and key is not null) (type: boolean) - Statistics: Num rows: 242 Data size: 968 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int) outputColumnNames: _col0 - Statistics: Num rows: 242 Data size: 968 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 83 Data size: 881 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: 242 Data size: 968 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator @@ -369,14 +366,14 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} 1 {KEY.reducesinkkey0} outputColumnNames: _col0, _col1 - Statistics: Num rows: 266 Data size: 1064 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 91 Data size: 969 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: int) outputColumnNames: _col0, _col1 - Statistics: Num rows: 266 Data size: 1064 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 91 Data size: 969 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 266 Data size: 1064 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 91 Data size: 969 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -409,36 +406,36 @@ STAGE PLANS: Map Operator Tree: TableScan alias: tab_part - 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 > 1) (type: boolean) - Statistics: Num rows: 484 Data size: 1936 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int) outputColumnNames: _col0 - Statistics: Num rows: 484 Data size: 1936 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 166 Data size: 1763 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: 484 Data size: 1936 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE Map 3 Map Operator Tree: TableScan alias: tab_part - 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 > 2) (type: boolean) - Statistics: Num rows: 484 Data size: 1936 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int) outputColumnNames: _col0 - Statistics: Num rows: 484 Data size: 1936 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 166 Data size: 1763 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: 484 Data size: 1936 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator @@ -448,14 +445,14 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} 1 {KEY.reducesinkkey0} outputColumnNames: _col0, _col1 - Statistics: Num rows: 532 Data size: 2129 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: int) outputColumnNames: _col0, _col1 - Statistics: Num rows: 532 Data size: 2129 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 532 Data size: 2129 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -488,36 +485,36 @@ STAGE PLANS: Map Operator Tree: TableScan alias: tab_part - 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 > 1) (type: boolean) - Statistics: Num rows: 484 Data size: 1936 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int) outputColumnNames: _col0 - Statistics: Num rows: 484 Data size: 1936 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 166 Data size: 1763 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: 484 Data size: 1936 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE Map 3 Map Operator Tree: TableScan alias: tab_part - 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 > 2) (type: boolean) - Statistics: Num rows: 484 Data size: 1936 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int) outputColumnNames: _col0 - Statistics: Num rows: 484 Data size: 1936 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 166 Data size: 1763 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: 484 Data size: 1936 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator @@ -527,14 +524,14 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} 1 {KEY.reducesinkkey0} outputColumnNames: _col0, _col1 - Statistics: Num rows: 532 Data size: 2129 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: int) outputColumnNames: _col0, _col1 - Statistics: Num rows: 532 Data size: 2129 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 532 Data size: 2129 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -566,37 +563,37 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 702 Data size: 2808 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 351 Data size: 1404 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 121 Data size: 1283 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: 351 Data size: 1404 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 121 Data size: 1283 Basic stats: COMPLETE Column stats: NONE Map 3 Map Operator Tree: TableScan alias: tab - Statistics: Num rows: 702 Data size: 2808 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 351 Data size: 1404 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 121 Data size: 1283 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int) outputColumnNames: key - Statistics: Num rows: 351 Data size: 1404 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 121 Data size: 1283 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: key (type: int) mode: hash outputColumnNames: _col0 - Statistics: Num rows: 351 Data size: 1404 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 121 Data size: 1283 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: 351 Data size: 1404 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 121 Data size: 1283 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator @@ -606,14 +603,14 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} 1 {KEY.reducesinkkey0} outputColumnNames: _col0, _col1 - Statistics: Num rows: 386 Data size: 1544 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 133 Data size: 1411 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: int) outputColumnNames: _col0, _col1 - Statistics: Num rows: 386 Data size: 1544 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 133 Data size: 1411 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 386 Data size: 1544 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 133 Data size: 1411 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -624,16 +621,16 @@ STAGE PLANS: keys: KEY._col0 (type: int) mode: mergepartial outputColumnNames: _col0 - Statistics: Num rows: 175 Data size: 700 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 60 Data size: 636 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int) outputColumnNames: _col0 - Statistics: Num rows: 175 Data size: 700 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 60 Data size: 636 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: 175 Data size: 700 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 60 Data size: 636 Basic stats: COMPLETE Column stats: NONE Stage: Stage-0 Fetch Operator @@ -661,38 +658,38 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 27 Data size: 2808 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: UDFToDouble(key) is not null (type: boolean) - Statistics: Num rows: 14 Data size: 1456 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 121 Data size: 1283 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: UDFToDouble(key) (type: double) sort order: + Map-reduce partition columns: UDFToDouble(key) (type: double) - Statistics: Num rows: 14 Data size: 1456 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 121 Data size: 1283 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Map 3 Map Operator Tree: TableScan alias: tab - Statistics: Num rows: 28 Data size: 2808 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: UDFToDouble(value) is not null (type: boolean) - Statistics: Num rows: 14 Data size: 1404 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 121 Data size: 1283 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: value (type: string) outputColumnNames: value - Statistics: Num rows: 14 Data size: 1404 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 121 Data size: 1283 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: value (type: string) mode: hash outputColumnNames: _col0 - Statistics: Num rows: 14 Data size: 1404 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 121 Data size: 1283 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: 14 Data size: 1404 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 121 Data size: 1283 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator @@ -702,14 +699,14 @@ STAGE PLANS: 0 {VALUE._col0} 1 {VALUE._col1} outputColumnNames: _col0, _col2 - Statistics: Num rows: 15 Data size: 1601 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 133 Data size: 1411 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col2 (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 15 Data size: 1601 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 133 Data size: 1411 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 15 Data size: 1601 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 133 Data size: 1411 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -720,16 +717,16 @@ STAGE PLANS: keys: KEY._col0 (type: string) mode: mergepartial outputColumnNames: _col0 - Statistics: Num rows: 7 Data size: 702 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 60 Data size: 636 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string) outputColumnNames: _col0 - Statistics: Num rows: 7 Data size: 702 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 60 Data size: 636 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: UDFToDouble(_col0) (type: double) sort order: + Map-reduce partition columns: UDFToDouble(_col0) (type: double) - Statistics: Num rows: 7 Data size: 702 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 60 Data size: 636 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string) Stage: Stage-0 diff --git ql/src/test/results/clientpositive/spark/bucketmapjoin1.q.out ql/src/test/results/clientpositive/spark/bucketmapjoin1.q.out index bea92f9..2524f6b 100644 --- ql/src/test/results/clientpositive/spark/bucketmapjoin1.q.out +++ ql/src/test/results/clientpositive/spark/bucketmapjoin1.q.out @@ -683,7 +683,6 @@ PREHOOK: Input: default@srcbucket_mapjoin PREHOOK: Input: default@srcbucket_mapjoin_part PREHOOK: Input: default@srcbucket_mapjoin_part@ds=2008-04-08 PREHOOK: Output: default@bucketmapjoin_tmp_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 bucketmapjoin_tmp_result select /*+mapjoin(b)*/ a.key, a.value, b.value from srcbucket_mapjoin a join srcbucket_mapjoin_part b @@ -710,7 +709,6 @@ select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_t PREHOOK: type: QUERY PREHOOK: Input: default@bucketmapjoin_tmp_result PREHOOK: Output: default@bucketmapjoin_hash_result_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 bucketmapjoin_hash_result_1 select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result POSTHOOK: type: QUERY @@ -728,7 +726,6 @@ PREHOOK: Input: default@srcbucket_mapjoin PREHOOK: Input: default@srcbucket_mapjoin_part PREHOOK: Input: default@srcbucket_mapjoin_part@ds=2008-04-08 PREHOOK: Output: default@bucketmapjoin_tmp_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 bucketmapjoin_tmp_result select /*+mapjoin(b)*/ a.key, a.value, b.value from srcbucket_mapjoin a join srcbucket_mapjoin_part b @@ -755,7 +752,6 @@ select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_t PREHOOK: type: QUERY PREHOOK: Input: default@bucketmapjoin_tmp_result PREHOOK: Output: default@bucketmapjoin_hash_result_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 bucketmapjoin_hash_result_2 select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result POSTHOOK: type: QUERY @@ -1030,8 +1026,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.bucketmapjoin_tmp_result numFiles 1 - numRows -1 - rawDataSize -1 + numRows 464 + rawDataSize 8519 serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1063,8 +1059,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.bucketmapjoin_tmp_result numFiles 1 - numRows -1 - rawDataSize -1 + numRows 464 + rawDataSize 8519 serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1086,7 +1082,6 @@ PREHOOK: Input: default@srcbucket_mapjoin PREHOOK: Input: default@srcbucket_mapjoin_part PREHOOK: Input: default@srcbucket_mapjoin_part@ds=2008-04-08 PREHOOK: Output: default@bucketmapjoin_tmp_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 bucketmapjoin_tmp_result select /*+mapjoin(a)*/ a.key, a.value, b.value from srcbucket_mapjoin a join srcbucket_mapjoin_part b @@ -1113,7 +1108,6 @@ select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_t PREHOOK: type: QUERY PREHOOK: Input: default@bucketmapjoin_tmp_result PREHOOK: Output: default@bucketmapjoin_hash_result_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 bucketmapjoin_hash_result_1 select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result POSTHOOK: type: QUERY @@ -1131,7 +1125,6 @@ PREHOOK: Input: default@srcbucket_mapjoin PREHOOK: Input: default@srcbucket_mapjoin_part PREHOOK: Input: default@srcbucket_mapjoin_part@ds=2008-04-08 PREHOOK: Output: default@bucketmapjoin_tmp_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 bucketmapjoin_tmp_result select /*+mapjoin(a)*/ a.key, a.value, b.value from srcbucket_mapjoin a join srcbucket_mapjoin_part b @@ -1158,7 +1151,6 @@ select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_t PREHOOK: type: QUERY PREHOOK: Input: default@bucketmapjoin_tmp_result PREHOOK: Output: default@bucketmapjoin_hash_result_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 bucketmapjoin_hash_result_2 select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result POSTHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/spark/bucketmapjoin13.q.out ql/src/test/results/clientpositive/spark/bucketmapjoin13.q.out index dc1b8cf..439f214 100644 --- ql/src/test/results/clientpositive/spark/bucketmapjoin13.q.out +++ ql/src/test/results/clientpositive/spark/bucketmapjoin13.q.out @@ -14,7 +14,6 @@ SELECT * FROM src PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@srcbucket_mapjoin_part_1@part=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: -- part=1 partition for srcbucket_mapjoin_part_1 is bucketed by 'value' INSERT OVERWRITE TABLE srcbucket_mapjoin_part_1 PARTITION (part='1') SELECT * FROM src @@ -37,7 +36,6 @@ SELECT * FROM src PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@srcbucket_mapjoin_part_1@part=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: -- part=2 partition for srcbucket_mapjoin_part_1 is bucketed by 'key' INSERT OVERWRITE TABLE srcbucket_mapjoin_part_1 PARTITION (part='2') SELECT * FROM src @@ -62,7 +60,6 @@ SELECT * FROM src PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@srcbucket_mapjoin_part_2@part=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: -- part=1 partition for srcbucket_mapjoin_part_2 is bucketed by 'key' INSERT OVERWRITE TABLE srcbucket_mapjoin_part_2 PARTITION (part='1') SELECT * FROM src @@ -138,17 +135,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - 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 GatherStats: false Filter Operator isSamplingPred: false 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 tag: 1 auto parallelism: true Path -> Alias: @@ -171,10 +168,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.srcbucket_mapjoin_part_2 numFiles 2 - numRows -1 + numRows 500 partition_columns part partition_columns.types string - rawDataSize -1 + rawDataSize 5312 serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -207,17 +204,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: a - Statistics: Num rows: 2906 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: key is not null (type: boolean) - 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 tag: 0 auto parallelism: true Path -> Alias: @@ -240,10 +237,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.srcbucket_mapjoin_part_1 numFiles 2 - numRows -1 + numRows 500 partition_columns part partition_columns.types string - rawDataSize -1 + rawDataSize 5312 serialization.ddl struct srcbucket_mapjoin_part_1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -287,10 +284,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.srcbucket_mapjoin_part_1 numFiles 2 - numRows -1 + numRows 500 partition_columns part partition_columns.types string - rawDataSize -1 + rawDataSize 5312 serialization.ddl struct srcbucket_mapjoin_part_1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -329,9 +326,9 @@ STAGE PLANS: condition expressions: 0 1 - Statistics: Num rows: 1598 Data size: 6393 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 1598 Data size: 6393 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash @@ -478,17 +475,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - 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 GatherStats: false Filter Operator isSamplingPred: false 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 tag: 1 auto parallelism: true Path -> Alias: @@ -511,10 +508,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.srcbucket_mapjoin_part_2 numFiles 2 - numRows -1 + numRows 500 partition_columns part partition_columns.types string - rawDataSize -1 + rawDataSize 5312 serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -547,17 +544,17 @@ 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 GatherStats: false Filter Operator isSamplingPred: false 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 tag: 0 auto parallelism: true Path -> Alias: @@ -580,10 +577,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.srcbucket_mapjoin_part_1 numFiles 2 - numRows -1 + numRows 500 partition_columns part partition_columns.types string - rawDataSize -1 + rawDataSize 5312 serialization.ddl struct srcbucket_mapjoin_part_1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -621,9 +618,9 @@ STAGE PLANS: condition expressions: 0 1 - Statistics: Num rows: 799 Data size: 3198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 799 Data size: 3198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash @@ -769,17 +766,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - 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 GatherStats: false Filter Operator isSamplingPred: false 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 tag: 1 auto parallelism: true Path -> Alias: @@ -802,10 +799,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.srcbucket_mapjoin_part_2 numFiles 2 - numRows -1 + numRows 500 partition_columns part partition_columns.types string - rawDataSize -1 + rawDataSize 5312 serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -838,17 +835,17 @@ 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 GatherStats: false Filter Operator isSamplingPred: false 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 tag: 0 auto parallelism: true Path -> Alias: @@ -871,10 +868,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.srcbucket_mapjoin_part_1 numFiles 2 - numRows -1 + numRows 500 partition_columns part partition_columns.types string - rawDataSize -1 + rawDataSize 5312 serialization.ddl struct srcbucket_mapjoin_part_1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -912,9 +909,9 @@ STAGE PLANS: condition expressions: 0 1 - Statistics: Num rows: 799 Data size: 3198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 799 Data size: 3198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash @@ -1062,17 +1059,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - 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 GatherStats: false Filter Operator isSamplingPred: false 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 tag: 1 auto parallelism: true Path -> Alias: @@ -1095,10 +1092,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.srcbucket_mapjoin_part_2 numFiles 2 - numRows -1 + numRows 500 partition_columns part partition_columns.types string - rawDataSize -1 + rawDataSize 5312 serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1131,17 +1128,17 @@ 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 GatherStats: false Filter Operator isSamplingPred: false 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 tag: 0 auto parallelism: true Path -> Alias: @@ -1164,10 +1161,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.srcbucket_mapjoin_part_1 numFiles 2 - numRows -1 + numRows 500 partition_columns part partition_columns.types string - rawDataSize -1 + rawDataSize 5312 serialization.ddl struct srcbucket_mapjoin_part_1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1205,9 +1202,9 @@ STAGE PLANS: condition expressions: 0 1 - Statistics: Num rows: 799 Data size: 3198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 799 Data size: 3198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash diff --git ql/src/test/results/clientpositive/spark/bucketmapjoin2.q.out ql/src/test/results/clientpositive/spark/bucketmapjoin2.q.out index 369b7fa..92c5fbd 100644 --- ql/src/test/results/clientpositive/spark/bucketmapjoin2.q.out +++ ql/src/test/results/clientpositive/spark/bucketmapjoin2.q.out @@ -391,7 +391,6 @@ PREHOOK: Input: default@srcbucket_mapjoin_part@ds=2008-04-08 PREHOOK: Input: default@srcbucket_mapjoin_part_2 PREHOOK: Input: default@srcbucket_mapjoin_part_2@ds=2008-04-08 PREHOOK: Output: default@bucketmapjoin_tmp_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 bucketmapjoin_tmp_result select /*+mapjoin(b)*/ a.key, a.value, b.value from srcbucket_mapjoin_part a join srcbucket_mapjoin_part_2 b @@ -419,7 +418,6 @@ select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_t PREHOOK: type: QUERY PREHOOK: Input: default@bucketmapjoin_tmp_result PREHOOK: Output: default@bucketmapjoin_hash_result_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 bucketmapjoin_hash_result_1 select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result POSTHOOK: type: QUERY @@ -438,7 +436,6 @@ PREHOOK: Input: default@srcbucket_mapjoin_part@ds=2008-04-08 PREHOOK: Input: default@srcbucket_mapjoin_part_2 PREHOOK: Input: default@srcbucket_mapjoin_part_2@ds=2008-04-08 PREHOOK: Output: default@bucketmapjoin_tmp_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 bucketmapjoin_tmp_result select /*+mapjoin(b)*/ a.key, a.value, b.value from srcbucket_mapjoin_part a join srcbucket_mapjoin_part_2 b @@ -466,7 +463,6 @@ select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_t PREHOOK: type: QUERY PREHOOK: Input: default@bucketmapjoin_tmp_result PREHOOK: Output: default@bucketmapjoin_hash_result_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 bucketmapjoin_hash_result_2 select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result POSTHOOK: type: QUERY @@ -746,8 +742,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.bucketmapjoin_tmp_result numFiles 1 - numRows -1 - rawDataSize -1 + numRows 564 + rawDataSize 10503 serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -779,8 +775,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.bucketmapjoin_tmp_result numFiles 1 - numRows -1 - rawDataSize -1 + numRows 564 + rawDataSize 10503 serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -803,7 +799,6 @@ PREHOOK: Input: default@srcbucket_mapjoin_part@ds=2008-04-08 PREHOOK: Input: default@srcbucket_mapjoin_part_2 PREHOOK: Input: default@srcbucket_mapjoin_part_2@ds=2008-04-08 PREHOOK: Output: default@bucketmapjoin_tmp_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 bucketmapjoin_tmp_result select /*+mapjoin(a)*/ a.key, a.value, b.value from srcbucket_mapjoin_part a join srcbucket_mapjoin_part_2 b @@ -831,7 +826,6 @@ select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_t PREHOOK: type: QUERY PREHOOK: Input: default@bucketmapjoin_tmp_result PREHOOK: Output: default@bucketmapjoin_hash_result_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 bucketmapjoin_hash_result_1 select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result POSTHOOK: type: QUERY @@ -850,7 +844,6 @@ PREHOOK: Input: default@srcbucket_mapjoin_part@ds=2008-04-08 PREHOOK: Input: default@srcbucket_mapjoin_part_2 PREHOOK: Input: default@srcbucket_mapjoin_part_2@ds=2008-04-08 PREHOOK: Output: default@bucketmapjoin_tmp_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 bucketmapjoin_tmp_result select /*+mapjoin(a)*/ a.key, a.value, b.value from srcbucket_mapjoin_part a join srcbucket_mapjoin_part_2 b @@ -878,7 +871,6 @@ select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_t PREHOOK: type: QUERY PREHOOK: Input: default@bucketmapjoin_tmp_result PREHOOK: Output: default@bucketmapjoin_hash_result_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 bucketmapjoin_hash_result_2 select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result POSTHOOK: type: QUERY @@ -1218,8 +1210,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.bucketmapjoin_tmp_result numFiles 1 - numRows -1 - rawDataSize -1 + numRows 564 + rawDataSize 10503 serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1251,8 +1243,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.bucketmapjoin_tmp_result numFiles 1 - numRows -1 - rawDataSize -1 + numRows 564 + rawDataSize 10503 serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1276,7 +1268,6 @@ PREHOOK: Input: default@srcbucket_mapjoin_part_2 PREHOOK: Input: default@srcbucket_mapjoin_part_2@ds=2008-04-08 PREHOOK: Input: default@srcbucket_mapjoin_part_2@ds=2008-04-09 PREHOOK: Output: default@bucketmapjoin_tmp_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 bucketmapjoin_tmp_result select /*+mapjoin(b)*/ a.key, a.value, b.value from srcbucket_mapjoin_part a join srcbucket_mapjoin_part_2 b @@ -1305,7 +1296,6 @@ select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_t PREHOOK: type: QUERY PREHOOK: Input: default@bucketmapjoin_tmp_result PREHOOK: Output: default@bucketmapjoin_hash_result_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 bucketmapjoin_hash_result_1 select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result POSTHOOK: type: QUERY @@ -1325,7 +1315,6 @@ PREHOOK: Input: default@srcbucket_mapjoin_part_2 PREHOOK: Input: default@srcbucket_mapjoin_part_2@ds=2008-04-08 PREHOOK: Input: default@srcbucket_mapjoin_part_2@ds=2008-04-09 PREHOOK: Output: default@bucketmapjoin_tmp_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 bucketmapjoin_tmp_result select /*+mapjoin(b)*/ a.key, a.value, b.value from srcbucket_mapjoin_part a join srcbucket_mapjoin_part_2 b @@ -1354,7 +1343,6 @@ select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_t PREHOOK: type: QUERY PREHOOK: Input: default@bucketmapjoin_tmp_result PREHOOK: Output: default@bucketmapjoin_hash_result_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 bucketmapjoin_hash_result_2 select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result POSTHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/spark/bucketmapjoin3.q.out ql/src/test/results/clientpositive/spark/bucketmapjoin3.q.out index 85ffbc9..28503d3 100644 --- ql/src/test/results/clientpositive/spark/bucketmapjoin3.q.out +++ ql/src/test/results/clientpositive/spark/bucketmapjoin3.q.out @@ -422,7 +422,6 @@ PREHOOK: Input: default@srcbucket_mapjoin_part@ds=2008-04-08 PREHOOK: Input: default@srcbucket_mapjoin_part_2 PREHOOK: Input: default@srcbucket_mapjoin_part_2@ds=2008-04-08 PREHOOK: Output: default@bucketmapjoin_tmp_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 bucketmapjoin_tmp_result select /*+mapjoin(b)*/ a.key, a.value, b.value from srcbucket_mapjoin_part_2 a join srcbucket_mapjoin_part b @@ -450,7 +449,6 @@ select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_t PREHOOK: type: QUERY PREHOOK: Input: default@bucketmapjoin_tmp_result PREHOOK: Output: default@bucketmapjoin_hash_result_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 bucketmapjoin_hash_result_1 select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result POSTHOOK: type: QUERY @@ -469,7 +467,6 @@ PREHOOK: Input: default@srcbucket_mapjoin_part@ds=2008-04-08 PREHOOK: Input: default@srcbucket_mapjoin_part_2 PREHOOK: Input: default@srcbucket_mapjoin_part_2@ds=2008-04-08 PREHOOK: Output: default@bucketmapjoin_tmp_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 bucketmapjoin_tmp_result select /*+mapjoin(b)*/ a.key, a.value, b.value from srcbucket_mapjoin_part_2 a join srcbucket_mapjoin_part b @@ -497,7 +494,6 @@ select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_t PREHOOK: type: QUERY PREHOOK: Input: default@bucketmapjoin_tmp_result PREHOOK: Output: default@bucketmapjoin_hash_result_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 bucketmapjoin_hash_result_2 select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result POSTHOOK: type: QUERY @@ -784,8 +780,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.bucketmapjoin_tmp_result numFiles 1 - numRows -1 - rawDataSize -1 + numRows 564 + rawDataSize 10503 serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -817,8 +813,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.bucketmapjoin_tmp_result numFiles 1 - numRows -1 - rawDataSize -1 + numRows 564 + rawDataSize 10503 serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -841,7 +837,6 @@ PREHOOK: Input: default@srcbucket_mapjoin_part@ds=2008-04-08 PREHOOK: Input: default@srcbucket_mapjoin_part_2 PREHOOK: Input: default@srcbucket_mapjoin_part_2@ds=2008-04-08 PREHOOK: Output: default@bucketmapjoin_tmp_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 bucketmapjoin_tmp_result select /*+mapjoin(a)*/ a.key, a.value, b.value from srcbucket_mapjoin_part_2 a join srcbucket_mapjoin_part b @@ -869,7 +864,6 @@ select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_t PREHOOK: type: QUERY PREHOOK: Input: default@bucketmapjoin_tmp_result PREHOOK: Output: default@bucketmapjoin_hash_result_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 bucketmapjoin_hash_result_2 select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result POSTHOOK: type: QUERY @@ -888,7 +882,6 @@ PREHOOK: Input: default@srcbucket_mapjoin_part@ds=2008-04-08 PREHOOK: Input: default@srcbucket_mapjoin_part_2 PREHOOK: Input: default@srcbucket_mapjoin_part_2@ds=2008-04-08 PREHOOK: Output: default@bucketmapjoin_tmp_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 bucketmapjoin_tmp_result select /*+mapjoin(a)*/ a.key, a.value, b.value from srcbucket_mapjoin_part_2 a join srcbucket_mapjoin_part b @@ -916,7 +909,6 @@ select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_t PREHOOK: type: QUERY PREHOOK: Input: default@bucketmapjoin_tmp_result PREHOOK: Output: default@bucketmapjoin_hash_result_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 bucketmapjoin_hash_result_2 select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result POSTHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/spark/bucketmapjoin4.q.out ql/src/test/results/clientpositive/spark/bucketmapjoin4.q.out index d11b9f4..e989172 100644 --- ql/src/test/results/clientpositive/spark/bucketmapjoin4.q.out +++ ql/src/test/results/clientpositive/spark/bucketmapjoin4.q.out @@ -395,7 +395,6 @@ on a.key=b.key PREHOOK: type: QUERY PREHOOK: Input: default@srcbucket_mapjoin PREHOOK: Output: default@bucketmapjoin_tmp_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 bucketmapjoin_tmp_result select /*+mapjoin(b)*/ a.key, a.value, b.value from srcbucket_mapjoin a join srcbucket_mapjoin b @@ -420,7 +419,6 @@ select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_t PREHOOK: type: QUERY PREHOOK: Input: default@bucketmapjoin_tmp_result PREHOOK: Output: default@bucketmapjoin_hash_result_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 bucketmapjoin_hash_result_1 select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result POSTHOOK: type: QUERY @@ -436,7 +434,6 @@ on a.key=b.key PREHOOK: type: QUERY PREHOOK: Input: default@srcbucket_mapjoin PREHOOK: Output: default@bucketmapjoin_tmp_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 bucketmapjoin_tmp_result select /*+mapjoin(b)*/ a.key, a.value, b.value from srcbucket_mapjoin a join srcbucket_mapjoin b @@ -461,7 +458,6 @@ select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_t PREHOOK: type: QUERY PREHOOK: Input: default@bucketmapjoin_tmp_result PREHOOK: Output: default@bucketmapjoin_hash_result_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 bucketmapjoin_hash_result_2 select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result POSTHOOK: type: QUERY @@ -724,8 +720,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.bucketmapjoin_tmp_result numFiles 1 - numRows -1 - rawDataSize -1 + numRows 464 + rawDataSize 8519 serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -757,8 +753,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.bucketmapjoin_tmp_result numFiles 1 - numRows -1 - rawDataSize -1 + numRows 464 + rawDataSize 8519 serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -778,7 +774,6 @@ on a.key=b.key PREHOOK: type: QUERY PREHOOK: Input: default@srcbucket_mapjoin PREHOOK: Output: default@bucketmapjoin_tmp_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 bucketmapjoin_tmp_result select /*+mapjoin(a)*/ a.key, a.value, b.value from srcbucket_mapjoin a join srcbucket_mapjoin b @@ -803,7 +798,6 @@ select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_t PREHOOK: type: QUERY PREHOOK: Input: default@bucketmapjoin_tmp_result PREHOOK: Output: default@bucketmapjoin_hash_result_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 bucketmapjoin_hash_result_1 select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result POSTHOOK: type: QUERY @@ -819,7 +813,6 @@ on a.key=b.key PREHOOK: type: QUERY PREHOOK: Input: default@srcbucket_mapjoin PREHOOK: Output: default@bucketmapjoin_tmp_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 bucketmapjoin_tmp_result select /*+mapjoin(a)*/ a.key, a.value, b.value from srcbucket_mapjoin a join srcbucket_mapjoin b @@ -844,7 +837,6 @@ select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_t PREHOOK: type: QUERY PREHOOK: Input: default@bucketmapjoin_tmp_result PREHOOK: Output: default@bucketmapjoin_hash_result_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 bucketmapjoin_hash_result_2 select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result POSTHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/spark/bucketmapjoin5.q.out ql/src/test/results/clientpositive/spark/bucketmapjoin5.q.out index 1bbc30c..2a8a3cc 100644 --- ql/src/test/results/clientpositive/spark/bucketmapjoin5.q.out +++ ql/src/test/results/clientpositive/spark/bucketmapjoin5.q.out @@ -501,7 +501,6 @@ PREHOOK: Input: default@srcbucket_mapjoin_part PREHOOK: Input: default@srcbucket_mapjoin_part@ds=2008-04-08 PREHOOK: Input: default@srcbucket_mapjoin_part@ds=2008-04-09 PREHOOK: Output: default@bucketmapjoin_tmp_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 bucketmapjoin_tmp_result select /*+mapjoin(a)*/ a.key, a.value, b.value from srcbucket_mapjoin a join srcbucket_mapjoin_part b @@ -529,7 +528,6 @@ select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_t PREHOOK: type: QUERY PREHOOK: Input: default@bucketmapjoin_tmp_result PREHOOK: Output: default@bucketmapjoin_hash_result_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 bucketmapjoin_hash_result_1 select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result POSTHOOK: type: QUERY @@ -548,7 +546,6 @@ PREHOOK: Input: default@srcbucket_mapjoin_part PREHOOK: Input: default@srcbucket_mapjoin_part@ds=2008-04-08 PREHOOK: Input: default@srcbucket_mapjoin_part@ds=2008-04-09 PREHOOK: Output: default@bucketmapjoin_tmp_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 bucketmapjoin_tmp_result select /*+mapjoin(a)*/ a.key, a.value, b.value from srcbucket_mapjoin a join srcbucket_mapjoin_part b @@ -576,7 +573,6 @@ select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_t PREHOOK: type: QUERY PREHOOK: Input: default@bucketmapjoin_tmp_result PREHOOK: Output: default@bucketmapjoin_hash_result_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 bucketmapjoin_hash_result_2 select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result POSTHOOK: type: QUERY @@ -892,8 +888,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.bucketmapjoin_tmp_result numFiles 1 - numRows -1 - rawDataSize -1 + numRows 928 + rawDataSize 17038 serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -925,8 +921,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.bucketmapjoin_tmp_result numFiles 1 - numRows -1 - rawDataSize -1 + numRows 928 + rawDataSize 17038 serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -949,7 +945,6 @@ PREHOOK: Input: default@srcbucket_mapjoin_part_2 PREHOOK: Input: default@srcbucket_mapjoin_part_2@ds=2008-04-08 PREHOOK: Input: default@srcbucket_mapjoin_part_2@ds=2008-04-09 PREHOOK: Output: default@bucketmapjoin_tmp_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 bucketmapjoin_tmp_result select /*+mapjoin(a)*/ a.key, a.value, b.value from srcbucket_mapjoin a join srcbucket_mapjoin_part_2 b @@ -977,7 +972,6 @@ select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_t PREHOOK: type: QUERY PREHOOK: Input: default@bucketmapjoin_tmp_result PREHOOK: Output: default@bucketmapjoin_hash_result_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 bucketmapjoin_hash_result_1 select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result POSTHOOK: type: QUERY @@ -996,7 +990,6 @@ PREHOOK: Input: default@srcbucket_mapjoin_part_2 PREHOOK: Input: default@srcbucket_mapjoin_part_2@ds=2008-04-08 PREHOOK: Input: default@srcbucket_mapjoin_part_2@ds=2008-04-09 PREHOOK: Output: default@bucketmapjoin_tmp_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 bucketmapjoin_tmp_result select /*+mapjoin(a)*/ a.key, a.value, b.value from srcbucket_mapjoin a join srcbucket_mapjoin_part_2 b @@ -1024,7 +1017,6 @@ select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_t PREHOOK: type: QUERY PREHOOK: Input: default@bucketmapjoin_tmp_result PREHOOK: Output: default@bucketmapjoin_hash_result_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 bucketmapjoin_hash_result_2 select sum(hash(key)), sum(hash(value1)), sum(hash(value2)) from bucketmapjoin_tmp_result POSTHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/spark/bucketmapjoin6.q.out ql/src/test/results/clientpositive/spark/bucketmapjoin6.q.out index 6f76566..198404b 100644 --- ql/src/test/results/clientpositive/spark/bucketmapjoin6.q.out +++ ql/src/test/results/clientpositive/spark/bucketmapjoin6.q.out @@ -22,7 +22,6 @@ PREHOOK: query: insert overwrite table tmp1 select * from src where key < 50 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@tmp1 -[Error 30017]: Skipping 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 tmp1 select * from src where key < 50 POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -33,7 +32,6 @@ PREHOOK: query: insert overwrite table tmp2 select * from src where key < 50 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@tmp2 -[Error 30017]: Skipping 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 tmp2 select * from src where key < 50 POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -55,7 +53,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@tmp1 PREHOOK: Input: default@tmp2 PREHOOK: Output: default@tmp3 -[Error 30017]: Skipping 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 tmp3 select /*+ MAPJOIN(l) */ i.a, i.b, l.b from tmp1 i join tmp2 l ON i.a = l.a 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/ctas.q.out ql/src/test/results/clientpositive/spark/ctas.q.out index 0ded266..d8ab800 100644 --- ql/src/test/results/clientpositive/spark/ctas.q.out +++ ql/src/test/results/clientpositive/spark/ctas.q.out @@ -107,7 +107,6 @@ PREHOOK: type: CREATETABLE_AS_SELECT PREHOOK: Input: default@src PREHOOK: Output: database:default PREHOOK: Output: default@nzhang_CTAS1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: create table nzhang_CTAS1 as select key k, value from src sort by k, value limit 10 POSTHOOK: type: CREATETABLE_AS_SELECT POSTHOOK: Input: default@src @@ -152,8 +151,8 @@ Table Type: MANAGED_TABLE Table Parameters: COLUMN_STATS_ACCURATE true numFiles 1 - numRows -1 - rawDataSize -1 + numRows 0 + rawDataSize 0 totalSize 106 #### A masked pattern was here #### @@ -256,7 +255,6 @@ PREHOOK: type: CREATETABLE_AS_SELECT PREHOOK: Input: default@src PREHOOK: Output: database:default PREHOOK: Output: default@nzhang_ctas2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: create table nzhang_ctas2 as select * from src sort by key, value limit 10 POSTHOOK: type: CREATETABLE_AS_SELECT POSTHOOK: Input: default@src @@ -301,8 +299,8 @@ Table Type: MANAGED_TABLE Table Parameters: COLUMN_STATS_ACCURATE true numFiles 1 - numRows -1 - rawDataSize -1 + numRows 10 + rawDataSize 96 totalSize 106 #### A masked pattern was here #### @@ -405,7 +403,6 @@ PREHOOK: type: CREATETABLE_AS_SELECT PREHOOK: Input: default@src PREHOOK: Output: database:default PREHOOK: Output: default@nzhang_ctas3 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: create table nzhang_ctas3 row format serde "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe" stored as RCFile as select key/2 half_key, concat(value, "_con") conb from src sort by half_key, conb limit 10 POSTHOOK: type: CREATETABLE_AS_SELECT POSTHOOK: Input: default@src @@ -450,8 +447,8 @@ Table Type: MANAGED_TABLE Table Parameters: COLUMN_STATS_ACCURATE true numFiles 1 - numRows -1 - rawDataSize -1 + numRows 10 + rawDataSize 120 totalSize 199 #### A masked pattern was here #### @@ -515,8 +512,8 @@ Table Type: MANAGED_TABLE Table Parameters: COLUMN_STATS_ACCURATE true numFiles 1 - numRows -1 - rawDataSize -1 + numRows 10 + rawDataSize 120 totalSize 199 #### A masked pattern was here #### @@ -620,7 +617,6 @@ PREHOOK: type: CREATETABLE_AS_SELECT PREHOOK: Input: default@src PREHOOK: Output: database:default PREHOOK: Output: default@nzhang_ctas4 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: create table nzhang_ctas4 row format delimited fields terminated by ',' stored as textfile as select key, value from src sort by key, value limit 10 POSTHOOK: type: CREATETABLE_AS_SELECT POSTHOOK: Input: default@src @@ -665,8 +661,8 @@ Table Type: MANAGED_TABLE Table Parameters: COLUMN_STATS_ACCURATE true numFiles 1 - numRows -1 - rawDataSize -1 + numRows 10 + rawDataSize 96 totalSize 106 #### A masked pattern was here #### @@ -887,7 +883,6 @@ PREHOOK: type: CREATETABLE_AS_SELECT PREHOOK: Input: default@src PREHOOK: Output: database:default PREHOOK: Output: default@nzhang_ctas5 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: create table nzhang_ctas5 row format delimited fields terminated by ',' lines terminated by '\012' stored as textfile as select key, value from src sort by key, value limit 10 POSTHOOK: type: CREATETABLE_AS_SELECT POSTHOOK: Input: default@src @@ -905,7 +900,6 @@ PREHOOK: query: insert overwrite table nzhang_ctas6 select key, value from src t PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@nzhang_ctas6 -[Error 30017]: Skipping 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_ctas6 select key, value from src tablesample (10 rows) POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -917,7 +911,6 @@ PREHOOK: type: CREATETABLE_AS_SELECT PREHOOK: Input: default@nzhang_ctas6 PREHOOK: Output: database:default PREHOOK: Output: default@nzhang_ctas7 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: create table nzhang_ctas7 as select key, `to` from nzhang_ctas6 POSTHOOK: type: CREATETABLE_AS_SELECT POSTHOOK: Input: default@nzhang_ctas6 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/infer_bucket_sort_convert_join.q.out ql/src/test/results/clientpositive/spark/infer_bucket_sort_convert_join.q.out index c99ff55..cd6779d 100644 --- ql/src/test/results/clientpositive/spark/infer_bucket_sort_convert_join.q.out +++ ql/src/test/results/clientpositive/spark/infer_bucket_sort_convert_join.q.out @@ -20,7 +20,6 @@ SELECT a.key, b.value FROM src a JOIN src b ON a.key = b.key PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@test_table@part=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: -- Tests a join which is converted to a map join, the output should be neither bucketed nor sorted INSERT OVERWRITE TABLE test_table PARTITION (part = '1') SELECT a.key, b.value FROM src a JOIN src b ON a.key = b.key @@ -55,8 +54,8 @@ Protect Mode: None Partition Parameters: COLUMN_STATS_ACCURATE true numFiles 1 - numRows -1 - rawDataSize -1 + numRows 1028 + rawDataSize 10968 totalSize 11996 #### A masked pattern was here #### @@ -78,7 +77,6 @@ SELECT a.key, b.value FROM src a JOIN src b ON a.key = b.key PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@test_table@part=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: -- This test tests the scenario when the mapper dies. So, create a conditional task for the mapjoin. -- Tests a join which is not converted to a map join, the output should be bucketed and sorted. @@ -115,8 +113,8 @@ Protect Mode: None Partition Parameters: COLUMN_STATS_ACCURATE true numFiles 1 - numRows -1 - rawDataSize -1 + numRows 1028 + rawDataSize 10968 totalSize 11996 #### A masked pattern was here #### 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/input13.q.out ql/src/test/results/clientpositive/spark/input13.q.out index 0ce60ac..03e53a7 100644 --- ql/src/test/results/clientpositive/spark/input13.q.out +++ ql/src/test/results/clientpositive/spark/input13.q.out @@ -179,9 +179,6 @@ PREHOOK: Output: default@dest1 PREHOOK: Output: default@dest2 PREHOOK: Output: default@dest3@ds=2008-04-08/hr=12 PREHOOK: Output: target/warehouse/dest4.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 -[Error 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/join28.q.out ql/src/test/results/clientpositive/spark/join28.q.out index 87d6819..0a7c1de 100644 --- ql/src/test/results/clientpositive/spark/join28.q.out +++ ql/src/test/results/clientpositive/spark/join28.q.out @@ -154,7 +154,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 subq.key1, z.value FROM diff --git ql/src/test/results/clientpositive/spark/join29.q.out ql/src/test/results/clientpositive/spark/join29.q.out index d5383d5..6da549d 100644 --- ql/src/test/results/clientpositive/spark/join29.q.out +++ ql/src/test/results/clientpositive/spark/join29.q.out @@ -172,7 +172,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 subq1.key, subq1.cnt, subq2.cnt FROM (select x.key, count(1) as cnt from src1 x group by x.key) subq1 JOIN 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/join30.q.out ql/src/test/results/clientpositive/spark/join30.q.out index 5c16622..8d7071e 100644 --- ql/src/test/results/clientpositive/spark/join30.q.out +++ ql/src/test/results/clientpositive/spark/join30.q.out @@ -127,7 +127,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, count(1) FROM src1 x JOIN src y ON (x.key = y.key) group by x.key POSTHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/spark/join31.q.out ql/src/test/results/clientpositive/spark/join31.q.out index 9193df9..8492326 100644 --- ql/src/test/results/clientpositive/spark/join31.q.out +++ ql/src/test/results/clientpositive/spark/join31.q.out @@ -198,7 +198,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 subq1.key, count(1) as cnt FROM (select x.key, count(1) as cnt from src1 x group by x.key) subq1 JOIN diff --git ql/src/test/results/clientpositive/spark/join32.q.out ql/src/test/results/clientpositive/spark/join32.q.out index 5aea3f3..43d5cce 100644 --- ql/src/test/results/clientpositive/spark/join32.q.out +++ ql/src/test/results/clientpositive/spark/join32.q.out @@ -415,7 +415,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 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/join32_lessSize.q.out ql/src/test/results/clientpositive/spark/join32_lessSize.q.out index bffd620..2c522f3 100644 --- ql/src/test/results/clientpositive/spark/join32_lessSize.q.out +++ ql/src/test/results/clientpositive/spark/join32_lessSize.q.out @@ -423,7 +423,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 x.key, z.value, y.value FROM src1 x JOIN src y ON (x.key = y.key) @@ -952,8 +951,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.dest_j1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 85 + rawDataSize 1600 serialization.ddl struct dest_j1 { string key, string value, string val2} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -985,8 +984,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.dest_j1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 85 + rawDataSize 1600 serialization.ddl struct dest_j1 { string key, string value, string val2} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1008,7 +1007,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 x.key, z.value, y.value FROM src w JOIN src1 x ON (x.value = w.value) @@ -1537,7 +1535,6 @@ PREHOOK: Input: default@src1 PREHOOK: Input: default@srcpart PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 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: INSERT OVERWRITE TABLE dest_j2 SELECT res.key, z.value, res.value FROM (select x.key, x.value from src1 x JOIN src y ON (x.key = y.key)) res @@ -2016,8 +2013,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.dest_j2 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 85 + rawDataSize 1600 serialization.ddl struct dest_j2 { string key, string value, string val2} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2049,8 +2046,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.dest_j2 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 85 + rawDataSize 1600 serialization.ddl struct dest_j2 { string key, string value, string val2} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -2073,7 +2070,6 @@ PREHOOK: Input: default@src1 PREHOOK: Input: default@srcpart PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 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: INSERT OVERWRITE TABLE dest_j2 SELECT res.key, z.value, res.value FROM (select x.key, x.value from src1 x LEFT OUTER JOIN src y ON (x.key = y.key)) res @@ -2327,7 +2323,6 @@ PREHOOK: Input: default@src1 PREHOOK: Input: default@srcpart PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 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: INSERT OVERWRITE TABLE dest_j2 SELECT res.key, x.value, res.value FROM (select x.key, x.value from src1 x JOIN src y ON (x.key = y.key)) res @@ -2569,7 +2564,6 @@ PREHOOK: Input: default@src1 PREHOOK: Input: default@srcpart PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 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: INSERT OVERWRITE TABLE dest_j2 SELECT res.key, y.value, res.value FROM (select x.key, x.value from src1 x JOIN src y ON (x.key = y.key)) res diff --git ql/src/test/results/clientpositive/spark/join33.q.out ql/src/test/results/clientpositive/spark/join33.q.out index 5aea3f3..43d5cce 100644 --- ql/src/test/results/clientpositive/spark/join33.q.out +++ ql/src/test/results/clientpositive/spark/join33.q.out @@ -415,7 +415,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 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/join34.q.out ql/src/test/results/clientpositive/spark/join34.q.out index 533c285..2680c22 100644 --- ql/src/test/results/clientpositive/spark/join34.q.out +++ ql/src/test/results/clientpositive/spark/join34.q.out @@ -438,7 +438,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 x.key, x.value, subq1.value FROM diff --git ql/src/test/results/clientpositive/spark/join35.q.out ql/src/test/results/clientpositive/spark/join35.q.out index 1750aec..8cc0ff6 100644 --- ql/src/test/results/clientpositive/spark/join35.q.out +++ ql/src/test/results/clientpositive/spark/join35.q.out @@ -504,7 +504,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 x.key, x.value, subq1.cnt FROM diff --git ql/src/test/results/clientpositive/spark/join36.q.out ql/src/test/results/clientpositive/spark/join36.q.out index 3c7e5e4..4828bd7 100644 --- ql/src/test/results/clientpositive/spark/join36.q.out +++ ql/src/test/results/clientpositive/spark/join36.q.out @@ -31,7 +31,6 @@ SELECT key, count(1) from src group by key PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@tmp1 -[Error 30017]: Skipping 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 tmp1 SELECT key, count(1) from src group by key POSTHOOK: type: QUERY @@ -44,7 +43,6 @@ SELECT key, count(1) from src group by key PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@tmp2 -[Error 30017]: Skipping 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 tmp2 SELECT key, count(1) from src group by key POSTHOOK: type: QUERY @@ -79,29 +77,29 @@ STAGE PLANS: Map Operator Tree: TableScan alias: y - Statistics: Num rows: 223 Data size: 1791 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 309 Data size: 1482 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 112 Data size: 899 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 155 Data size: 743 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: 112 Data size: 899 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 155 Data size: 743 Basic stats: COMPLETE Column stats: NONE value expressions: cnt (type: int) Map 3 Map Operator Tree: TableScan alias: x - Statistics: Num rows: 223 Data size: 1791 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 309 Data size: 1482 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 112 Data size: 899 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 155 Data size: 743 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: 112 Data size: 899 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 155 Data size: 743 Basic stats: COMPLETE Column stats: NONE value expressions: cnt (type: int) Reducer 2 Reduce Operator Tree: @@ -112,14 +110,14 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 {VALUE._col0} outputColumnNames: _col0, _col1, _col6 - Statistics: Num rows: 123 Data size: 988 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 170 Data size: 817 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: int), _col6 (type: int) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 123 Data size: 988 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 170 Data size: 817 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 123 Data size: 988 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 170 Data size: 817 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -149,7 +147,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@tmp1 PREHOOK: Input: default@tmp2 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.cnt, y.cnt FROM tmp1 x JOIN tmp2 y ON (x.key = y.key) diff --git ql/src/test/results/clientpositive/spark/join37.q.out ql/src/test/results/clientpositive/spark/join37.q.out index 1e5697b..6d402ac 100644 --- ql/src/test/results/clientpositive/spark/join37.q.out +++ ql/src/test/results/clientpositive/spark/join37.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/join38.q.out ql/src/test/results/clientpositive/spark/join38.q.out index cef8a84..6ec8c6b 100644 --- ql/src/test/results/clientpositive/spark/join38.q.out +++ ql/src/test/results/clientpositive/spark/join38.q.out @@ -10,7 +10,6 @@ PREHOOK: query: insert overwrite table tmp select key, cast(key + 1 as int), key PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@tmp -[Error 30017]: Skipping 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 select key, cast(key + 1 as int), key +2, key+3, key+4, cast(key+5 as int), key+6, key+7, key+8, key+9, key+10, cast(key+11 as int) from src where key = 100 POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -65,7 +64,7 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 1 Data size: 128 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 126 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (col11 is not null and (col11 = 111)) (type: boolean) Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE diff --git ql/src/test/results/clientpositive/spark/join39.q.out ql/src/test/results/clientpositive/spark/join39.q.out index 892a722..80bcc47 100644 --- ql/src/test/results/clientpositive/spark/join39.q.out +++ ql/src/test/results/clientpositive/spark/join39.q.out @@ -107,7 +107,6 @@ FROM src x left outer JOIN (select * from src where key <= 100) y ON (x.key = y. 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: INSERT OVERWRITE TABLE dest_j1 SELECT /*+ MAPJOIN(y) */ x.key, x.value, y.key, y.value FROM src x left outer JOIN (select * from src where key <= 100) y ON (x.key = y.key) 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/join41.q.out ql/src/test/results/clientpositive/spark/join41.q.out index d7c762d..cbc6e7a 100644 --- ql/src/test/results/clientpositive/spark/join41.q.out +++ ql/src/test/results/clientpositive/spark/join41.q.out @@ -3,7 +3,6 @@ PREHOOK: type: CREATETABLE_AS_SELECT PREHOOK: Input: default@src PREHOOK: Output: database:default PREHOOK: Output: default@s1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: create table s1 as select * from src where key = 0 POSTHOOK: type: CREATETABLE_AS_SELECT POSTHOOK: Input: default@src @@ -30,26 +29,26 @@ STAGE PLANS: Map Operator Tree: TableScan alias: src2 - Statistics: Num rows: 0 Data size: 24 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 3 Data size: 21 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (key > 10) (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: string) sort order: + Map-reduce partition columns: key (type: string) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Map 3 Map Operator Tree: TableScan alias: src1 - Statistics: Num rows: 0 Data size: 24 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 3 Data size: 21 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: string) sort order: + Map-reduce partition columns: key (type: string) - Statistics: Num rows: 0 Data size: 24 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 3 Data size: 21 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Reducer 2 Reduce Operator Tree: @@ -60,14 +59,14 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 {KEY.reducesinkkey0} {VALUE._col0} outputColumnNames: _col0, _col1, _col5, _col6 - Statistics: Num rows: 0 Data size: 26 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 3 Data size: 23 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 0 Data size: 26 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 3 Data size: 23 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 0 Data size: 26 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 3 Data size: 23 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -113,26 +112,26 @@ STAGE PLANS: Map Operator Tree: TableScan alias: src2 - Statistics: Num rows: 0 Data size: 24 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 3 Data size: 21 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (key > 10) (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: string) sort order: + Map-reduce partition columns: key (type: string) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Map 3 Map Operator Tree: TableScan alias: src1 - Statistics: Num rows: 0 Data size: 24 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 3 Data size: 21 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: string) sort order: + Map-reduce partition columns: key (type: string) - Statistics: Num rows: 0 Data size: 24 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 3 Data size: 21 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Reducer 2 Reduce Operator Tree: @@ -143,14 +142,14 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 {KEY.reducesinkkey0} {VALUE._col0} outputColumnNames: _col0, _col1, _col5, _col6 - Statistics: Num rows: 0 Data size: 26 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 3 Data size: 23 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 0 Data size: 26 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 3 Data size: 23 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 0 Data size: 26 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 3 Data size: 23 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/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_filters_overlap.q.out ql/src/test/results/clientpositive/spark/join_filters_overlap.q.out index 78c2b69..a153b00 100644 --- ql/src/test/results/clientpositive/spark/join_filters_overlap.q.out +++ ql/src/test/results/clientpositive/spark/join_filters_overlap.q.out @@ -6,7 +6,6 @@ PREHOOK: type: CREATETABLE_AS_SELECT PREHOOK: Input: default@src PREHOOK: Output: database:default PREHOOK: Output: default@a -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: -- SORT_QUERY_RESULTS -- HIVE-3411 Filter predicates on outer join overlapped on single alias is not handled properly @@ -108,17 +107,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 2 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: (value = 50) (type: boolean) - Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 6 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: 10 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE tag: 1 value expressions: value (type: int) auto parallelism: true @@ -139,8 +138,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.a numFiles 1 - numRows -1 - rawDataSize -1 + numRows 3 + rawDataSize 18 serialization.ddl struct a { i32 key, i32 value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -159,8 +158,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.a numFiles 1 - numRows -1 - rawDataSize -1 + numRows 3 + rawDataSize 18 serialization.ddl struct a { i32 key, i32 value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -175,17 +174,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: c - Statistics: Num rows: 2 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: (value = 60) (type: boolean) - Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 6 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: 10 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE tag: 2 value expressions: value (type: int) auto parallelism: true @@ -206,8 +205,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.a numFiles 1 - numRows -1 - rawDataSize -1 + numRows 3 + rawDataSize 18 serialization.ddl struct a { i32 key, i32 value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -226,8 +225,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.a numFiles 1 - numRows -1 - rawDataSize -1 + numRows 3 + rawDataSize 18 serialization.ddl struct a { i32 key, i32 value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -242,13 +241,13 @@ STAGE PLANS: Map Operator Tree: TableScan alias: a - Statistics: Num rows: 2 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE GatherStats: false Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - Statistics: Num rows: 2 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE tag: 0 value expressions: value (type: int) auto parallelism: true @@ -269,8 +268,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.a numFiles 1 - numRows -1 - rawDataSize -1 + numRows 3 + rawDataSize 18 serialization.ddl struct a { i32 key, i32 value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -289,8 +288,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.a numFiles 1 - numRows -1 - rawDataSize -1 + numRows 3 + rawDataSize 18 serialization.ddl struct a { i32 key, i32 value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -319,17 +318,17 @@ STAGE PLANS: 1 2 outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11 - Statistics: Num rows: 4 Data size: 46 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 39 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: int), _col5 (type: int), _col6 (type: int), _col10 (type: int), _col11 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 4 Data size: 46 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 39 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: 46 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 39 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -467,13 +466,13 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 2 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE GatherStats: false Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - Statistics: Num rows: 2 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE tag: 1 value expressions: value (type: int) auto parallelism: true @@ -494,8 +493,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.a numFiles 1 - numRows -1 - rawDataSize -1 + numRows 3 + rawDataSize 18 serialization.ddl struct a { i32 key, i32 value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -514,8 +513,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.a numFiles 1 - numRows -1 - rawDataSize -1 + numRows 3 + rawDataSize 18 serialization.ddl struct a { i32 key, i32 value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -530,17 +529,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: c - Statistics: Num rows: 2 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: (value = 60) (type: boolean) - Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 6 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: 10 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE tag: 2 value expressions: value (type: int) auto parallelism: true @@ -561,8 +560,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.a numFiles 1 - numRows -1 - rawDataSize -1 + numRows 3 + rawDataSize 18 serialization.ddl struct a { i32 key, i32 value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -581,8 +580,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.a numFiles 1 - numRows -1 - rawDataSize -1 + numRows 3 + rawDataSize 18 serialization.ddl struct a { i32 key, i32 value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -597,17 +596,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: a - Statistics: Num rows: 2 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: (value = 50) (type: boolean) - Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 6 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: 10 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE tag: 0 value expressions: value (type: int) auto parallelism: true @@ -628,8 +627,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.a numFiles 1 - numRows -1 - rawDataSize -1 + numRows 3 + rawDataSize 18 serialization.ddl struct a { i32 key, i32 value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -648,8 +647,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.a numFiles 1 - numRows -1 - rawDataSize -1 + numRows 3 + rawDataSize 18 serialization.ddl struct a { i32 key, i32 value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -678,17 +677,17 @@ STAGE PLANS: 1 {(VALUE._col0 = 50)} {(VALUE._col0 = 60)} 2 outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11 - Statistics: Num rows: 4 Data size: 46 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 39 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: int), _col5 (type: int), _col6 (type: int), _col10 (type: int), _col11 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 4 Data size: 46 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 39 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: 46 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 39 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -840,13 +839,13 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 2 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE GatherStats: false Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - Statistics: Num rows: 2 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE tag: 1 value expressions: value (type: int) auto parallelism: true @@ -867,8 +866,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.a numFiles 1 - numRows -1 - rawDataSize -1 + numRows 3 + rawDataSize 18 serialization.ddl struct a { i32 key, i32 value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -887,8 +886,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.a numFiles 1 - numRows -1 - rawDataSize -1 + numRows 3 + rawDataSize 18 serialization.ddl struct a { i32 key, i32 value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -903,17 +902,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: c - Statistics: Num rows: 2 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: (value = 60) (type: boolean) - Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 6 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: 10 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE tag: 2 value expressions: value (type: int) auto parallelism: true @@ -934,8 +933,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.a numFiles 1 - numRows -1 - rawDataSize -1 + numRows 3 + rawDataSize 18 serialization.ddl struct a { i32 key, i32 value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -954,8 +953,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.a numFiles 1 - numRows -1 - rawDataSize -1 + numRows 3 + rawDataSize 18 serialization.ddl struct a { i32 key, i32 value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -970,17 +969,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: a - Statistics: Num rows: 2 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: (value = 50) (type: boolean) - Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 6 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: 10 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE tag: 0 value expressions: value (type: int) auto parallelism: true @@ -1001,8 +1000,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.a numFiles 1 - numRows -1 - rawDataSize -1 + numRows 3 + rawDataSize 18 serialization.ddl struct a { i32 key, i32 value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1021,8 +1020,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.a numFiles 1 - numRows -1 - rawDataSize -1 + numRows 3 + rawDataSize 18 serialization.ddl struct a { i32 key, i32 value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1051,17 +1050,17 @@ STAGE PLANS: 1 {(VALUE._col0 = 50)} {(VALUE._col0 > 10)} {(VALUE._col0 = 60)} {(VALUE._col0 > 20)} 2 outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11 - Statistics: Num rows: 4 Data size: 46 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 39 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: int), _col5 (type: int), _col6 (type: int), _col10 (type: int), _col11 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 4 Data size: 46 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 39 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: 46 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 39 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -1227,17 +1226,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: d - Statistics: Num rows: 2 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: (value = 40) (type: boolean) - Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 6 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: 10 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE tag: 3 value expressions: value (type: int) auto parallelism: true @@ -1258,8 +1257,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.a numFiles 1 - numRows -1 - rawDataSize -1 + numRows 3 + rawDataSize 18 serialization.ddl struct a { i32 key, i32 value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1278,8 +1277,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.a numFiles 1 - numRows -1 - rawDataSize -1 + numRows 3 + rawDataSize 18 serialization.ddl struct a { i32 key, i32 value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1294,13 +1293,13 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 2 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE GatherStats: false Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - Statistics: Num rows: 2 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE tag: 1 value expressions: value (type: int) auto parallelism: true @@ -1321,8 +1320,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.a numFiles 1 - numRows -1 - rawDataSize -1 + numRows 3 + rawDataSize 18 serialization.ddl struct a { i32 key, i32 value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1341,8 +1340,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.a numFiles 1 - numRows -1 - rawDataSize -1 + numRows 3 + rawDataSize 18 serialization.ddl struct a { i32 key, i32 value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1357,17 +1356,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: c - Statistics: Num rows: 2 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: (value = 60) (type: boolean) - Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 6 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: 10 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE tag: 2 value expressions: value (type: int) auto parallelism: true @@ -1388,8 +1387,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.a numFiles 1 - numRows -1 - rawDataSize -1 + numRows 3 + rawDataSize 18 serialization.ddl struct a { i32 key, i32 value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1408,8 +1407,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.a numFiles 1 - numRows -1 - rawDataSize -1 + numRows 3 + rawDataSize 18 serialization.ddl struct a { i32 key, i32 value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1424,13 +1423,13 @@ STAGE PLANS: Map Operator Tree: TableScan alias: a - Statistics: Num rows: 2 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE GatherStats: false Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - Statistics: Num rows: 2 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE tag: 0 value expressions: value (type: int) auto parallelism: true @@ -1451,8 +1450,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.a numFiles 1 - numRows -1 - rawDataSize -1 + numRows 3 + rawDataSize 18 serialization.ddl struct a { i32 key, i32 value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1471,8 +1470,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.a numFiles 1 - numRows -1 - rawDataSize -1 + numRows 3 + rawDataSize 18 serialization.ddl struct a { i32 key, i32 value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1505,17 +1504,17 @@ STAGE PLANS: 2 3 outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11, _col15, _col16 - Statistics: Num rows: 6 Data size: 69 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 9 Data size: 59 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: int), _col5 (type: int), _col6 (type: int), _col10 (type: int), _col11 (type: int), _col15 (type: int), _col16 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 6 Data size: 69 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 9 Data size: 59 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: 69 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 9 Data size: 59 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### table: input format: org.apache.hadoop.mapred.TextInputFormat @@ -1672,17 +1671,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: d - Statistics: Num rows: 2 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: (value = 40) (type: boolean) - Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 6 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: 10 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE tag: 3 value expressions: value (type: int) auto parallelism: true @@ -1703,8 +1702,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.a numFiles 1 - numRows -1 - rawDataSize -1 + numRows 3 + rawDataSize 18 serialization.ddl struct a { i32 key, i32 value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1723,8 +1722,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.a numFiles 1 - numRows -1 - rawDataSize -1 + numRows 3 + rawDataSize 18 serialization.ddl struct a { i32 key, i32 value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1739,17 +1738,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 2 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: (value = 50) (type: boolean) - Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 6 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: 10 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE tag: 1 value expressions: value (type: int) auto parallelism: true @@ -1770,8 +1769,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.a numFiles 1 - numRows -1 - rawDataSize -1 + numRows 3 + rawDataSize 18 serialization.ddl struct a { i32 key, i32 value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1790,8 +1789,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.a numFiles 1 - numRows -1 - rawDataSize -1 + numRows 3 + rawDataSize 18 serialization.ddl struct a { i32 key, i32 value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1806,17 +1805,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: c - Statistics: Num rows: 2 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: (value = 60) (type: boolean) - Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 6 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: 10 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE tag: 2 value expressions: value (type: int) auto parallelism: true @@ -1837,8 +1836,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.a numFiles 1 - numRows -1 - rawDataSize -1 + numRows 3 + rawDataSize 18 serialization.ddl struct a { i32 key, i32 value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1857,8 +1856,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.a numFiles 1 - numRows -1 - rawDataSize -1 + numRows 3 + rawDataSize 18 serialization.ddl struct a { i32 key, i32 value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1873,13 +1872,13 @@ STAGE PLANS: Map Operator Tree: TableScan alias: a - Statistics: Num rows: 2 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE GatherStats: false Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - Statistics: Num rows: 2 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE tag: 0 value expressions: value (type: int) auto parallelism: true @@ -1900,8 +1899,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.a numFiles 1 - numRows -1 - rawDataSize -1 + numRows 3 + rawDataSize 18 serialization.ddl struct a { i32 key, i32 value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1920,8 +1919,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.a numFiles 1 - numRows -1 - rawDataSize -1 + numRows 3 + rawDataSize 18 serialization.ddl struct a { i32 key, i32 value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -1953,17 +1952,17 @@ STAGE PLANS: 2 3 outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11, _col15, _col16 - Statistics: Num rows: 6 Data size: 69 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 9 Data size: 59 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: int), _col5 (type: int), _col6 (type: int), _col10 (type: int), _col11 (type: int), _col15 (type: int), _col16 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Statistics: Num rows: 6 Data size: 69 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 9 Data size: 59 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: 69 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 9 Data size: 59 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/join_map_ppr.q.out ql/src/test/results/clientpositive/spark/join_map_ppr.q.out index 4ee6b8d..be54790 100644 --- ql/src/test/results/clientpositive/spark/join_map_ppr.q.out +++ ql/src/test/results/clientpositive/spark/join_map_ppr.q.out @@ -402,7 +402,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) @@ -552,7 +551,6 @@ PREHOOK: query: INSERT OVERWRITE TABLE src_copy select key, value from src PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@src_copy -[Error 30017]: Skipping 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_copy select key, value from src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -563,7 +561,6 @@ PREHOOK: query: INSERT OVERWRITE TABLE src1_copy select key, value from src1 PREHOOK: type: QUERY PREHOOK: Input: default@src1 PREHOOK: Output: default@src1_copy -[Error 30017]: Skipping 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_copy select key, value from src1 POSTHOOK: type: QUERY POSTHOOK: Input: default@src1 @@ -749,17 +746,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: y - 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: UDFToDouble(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: UDFToDouble(key) (type: double) sort order: + Map-reduce partition columns: UDFToDouble(key) (type: double) - 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: 1 value expressions: value (type: string) auto parallelism: true @@ -780,8 +777,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.src_copy numFiles 1 - numRows -1 - rawDataSize -1 + numRows 500 + rawDataSize 5312 serialization.ddl struct src_copy { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -800,8 +797,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.src_copy numFiles 1 - numRows -1 - rawDataSize -1 + numRows 500 + rawDataSize 5312 serialization.ddl struct src_copy { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -816,17 +813,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: x - Statistics: Num rows: 2 Data size: 216 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: UDFToDouble(key) is not null (type: boolean) - Statistics: Num rows: 1 Data size: 108 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: UDFToDouble(key) (type: double) sort order: + Map-reduce partition columns: UDFToDouble(key) (type: double) - Statistics: Num rows: 1 Data size: 108 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE tag: 0 value expressions: key (type: string) auto parallelism: true @@ -847,8 +844,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.src1_copy numFiles 1 - numRows -1 - rawDataSize -1 + numRows 25 + rawDataSize 191 serialization.ddl struct src1_copy { string key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -867,8 +864,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.src1_copy numFiles 1 - numRows -1 - rawDataSize -1 + numRows 25 + rawDataSize 191 serialization.ddl struct src1_copy { string key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -915,8 +912,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.dest_j1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 107 + rawDataSize 2018 serialization.ddl struct dest_j1 { string key, string value, string val2} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -948,8 +945,8 @@ STAGE PLANS: #### A masked pattern was here #### name default.dest_j1 numFiles 1 - numRows -1 - rawDataSize -1 + numRows 107 + rawDataSize 2018 serialization.ddl struct dest_j1 { string key, string value, string val2} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -973,7 +970,6 @@ PREHOOK: Input: default@src_copy 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_copy x JOIN src_copy y ON (x.key = y.key) 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/join_rc.q.out ql/src/test/results/clientpositive/spark/join_rc.q.out index 503d785..12cc876 100644 --- ql/src/test/results/clientpositive/spark/join_rc.q.out +++ ql/src/test/results/clientpositive/spark/join_rc.q.out @@ -18,7 +18,6 @@ PREHOOK: query: insert overwrite table join_rc1 select * from src PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@join_rc1 -[Error 30017]: Skipping 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 join_rc1 select * from src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -29,7 +28,6 @@ PREHOOK: query: insert overwrite table join_rc2 select * from src PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@join_rc2 -[Error 30017]: Skipping 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 join_rc2 select * from src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -59,29 +57,29 @@ STAGE PLANS: Map Operator Tree: TableScan alias: join_rc2 - Statistics: Num rows: 26 Data size: 5293 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 500 Data size: 4812 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 13 Data size: 2646 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2406 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: string) sort order: + Map-reduce partition columns: key (type: string) - Statistics: Num rows: 13 Data size: 2646 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2406 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Map 3 Map Operator Tree: TableScan alias: join_rc1 - Statistics: Num rows: 52 Data size: 5293 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 500 Data size: 4812 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 26 Data size: 2646 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2406 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: string) sort order: + Map-reduce partition columns: key (type: string) - Statistics: Num rows: 26 Data size: 2646 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2406 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator @@ -91,14 +89,14 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} 1 {VALUE._col0} outputColumnNames: _col0, _col6 - Statistics: Num rows: 28 Data size: 2910 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2646 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col6 (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 28 Data size: 2910 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2646 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 28 Data size: 2910 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2646 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/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/mapjoin_decimal.q.out ql/src/test/results/clientpositive/spark/mapjoin_decimal.q.out index 5e0a9e8..d150e97 100644 --- ql/src/test/results/clientpositive/spark/mapjoin_decimal.q.out +++ ql/src/test/results/clientpositive/spark/mapjoin_decimal.q.out @@ -50,7 +50,6 @@ PREHOOK: query: INSERT INTO TABLE t1 select dec from over1k PREHOOK: type: QUERY PREHOOK: Input: default@over1k 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: INSERT INTO TABLE t1 select dec from over1k POSTHOOK: type: QUERY POSTHOOK: Input: default@over1k @@ -68,7 +67,6 @@ PREHOOK: query: INSERT INTO TABLE t2 select dec from over1k PREHOOK: type: QUERY PREHOOK: Input: default@over1k 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: INSERT INTO TABLE t2 select dec from over1k POSTHOOK: type: QUERY POSTHOOK: Input: default@over1k @@ -95,28 +93,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t2 - Statistics: Num rows: 12 Data size: 1359 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1049 Data size: 117488 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: dec is not null (type: boolean) - Statistics: Num rows: 6 Data size: 679 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 525 Data size: 58800 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: dec (type: decimal(4,0)) sort order: + Map-reduce partition columns: dec (type: decimal(4,0)) - Statistics: Num rows: 6 Data size: 679 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 525 Data size: 58800 Basic stats: COMPLETE Column stats: NONE Map 3 Map Operator Tree: TableScan alias: t1 - Statistics: Num rows: 21 Data size: 2422 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1049 Data size: 117488 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: dec is not null (type: boolean) - Statistics: Num rows: 11 Data size: 1268 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 525 Data size: 58800 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: dec (type: decimal(4,2)) sort order: + Map-reduce partition columns: dec (type: decimal(4,2)) - Statistics: Num rows: 11 Data size: 1268 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 525 Data size: 58800 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator @@ -126,14 +124,14 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} 1 {KEY.reducesinkkey0} outputColumnNames: _col0, _col4 - Statistics: Num rows: 12 Data size: 1394 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 577 Data size: 64680 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: decimal(4,2)), _col4 (type: decimal(4,0)) outputColumnNames: _col0, _col1 - Statistics: Num rows: 12 Data size: 1394 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 577 Data size: 64680 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 12 Data size: 1394 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 577 Data size: 64680 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/mapjoin_hook.q.out ql/src/test/results/clientpositive/spark/mapjoin_hook.q.out index 8b1c223..c475f2a 100644 --- ql/src/test/results/clientpositive/spark/mapjoin_hook.q.out +++ ql/src/test/results/clientpositive/spark/mapjoin_hook.q.out @@ -12,7 +12,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Input: default@src1 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 [MapJoinCounter PostHook] COMMON_JOIN: 0 HINTED_MAPJOIN: 0 HINTED_MAPJOIN_LOCAL: 0 CONVERTED_MAPJOIN: 0 CONVERTED_MAPJOIN_LOCAL: 0 BACKUP_COMMON_JOIN: 0 RUN: Stage-1:MAPRED RUN: Stage-2:DEPENDENCY_COLLECTION @@ -23,7 +22,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 [MapJoinCounter PostHook] COMMON_JOIN: 0 HINTED_MAPJOIN: 0 HINTED_MAPJOIN_LOCAL: 0 CONVERTED_MAPJOIN: 0 CONVERTED_MAPJOIN_LOCAL: 0 BACKUP_COMMON_JOIN: 0 RUN: Stage-1:MAPRED RUN: Stage-2:DEPENDENCY_COLLECTION @@ -40,7 +38,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@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 [MapJoinCounter PostHook] COMMON_JOIN: 0 HINTED_MAPJOIN: 0 HINTED_MAPJOIN_LOCAL: 0 CONVERTED_MAPJOIN: 0 CONVERTED_MAPJOIN_LOCAL: 0 BACKUP_COMMON_JOIN: 0 RUN: Stage-1:MAPRED RUN: Stage-2:DEPENDENCY_COLLECTION @@ -51,7 +48,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 [MapJoinCounter PostHook] COMMON_JOIN: 0 HINTED_MAPJOIN: 0 HINTED_MAPJOIN_LOCAL: 0 CONVERTED_MAPJOIN: 0 CONVERTED_MAPJOIN_LOCAL: 0 BACKUP_COMMON_JOIN: 0 RUN: Stage-1:MAPRED RUN: Stage-2:DEPENDENCY_COLLECTION diff --git ql/src/test/results/clientpositive/spark/mapjoin_memcheck.q.out ql/src/test/results/clientpositive/spark/mapjoin_memcheck.q.out index b04ccec..e667137 100644 --- ql/src/test/results/clientpositive/spark/mapjoin_memcheck.q.out +++ ql/src/test/results/clientpositive/spark/mapjoin_memcheck.q.out @@ -14,7 +14,6 @@ PREHOOK: query: insert into table src0 select * from src where src.key < 10 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@src0 -[Error 30017]: Skipping 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 src0 select * from src where src.key < 10 POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -44,29 +43,29 @@ STAGE PLANS: Map Operator Tree: TableScan alias: src2 - 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: string) sort order: + Map-reduce partition columns: key (type: string) - 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: src1 - 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: string) sort order: + Map-reduce partition columns: key (type: string) - 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: @@ -77,14 +76,14 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 {KEY.reducesinkkey0} {VALUE._col0} outputColumnNames: _col0, _col1, _col5, _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: string), _col1 (type: string), _col5 (type: string), _col6 (type: string) outputColumnNames: _col0, _col1, _col2, _col3 - 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 diff --git ql/src/test/results/clientpositive/spark/mapjoin_test_outer.q.out ql/src/test/results/clientpositive/spark/mapjoin_test_outer.q.out index 19d03a0..4595431 100644 --- ql/src/test/results/clientpositive/spark/mapjoin_test_outer.q.out +++ ql/src/test/results/clientpositive/spark/mapjoin_test_outer.q.out @@ -14,7 +14,6 @@ PREHOOK: query: insert overwrite table dest_1 select * from src1 order by src1.v PREHOOK: type: QUERY PREHOOK: Input: default@src1 PREHOOK: Output: default@dest_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 dest_1 select * from src1 order by src1.value limit 8 POSTHOOK: type: QUERY POSTHOOK: Input: default@src1 @@ -25,7 +24,6 @@ PREHOOK: query: insert into table dest_1 select "333444","555666" from src1 limi PREHOOK: type: QUERY PREHOOK: Input: default@src1 PREHOOK: Output: default@dest_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 dest_1 select "333444","555666" from src1 limit 1 POSTHOOK: type: QUERY POSTHOOK: Input: default@src1 @@ -44,7 +42,6 @@ PREHOOK: query: insert into table dest_2 select * from dest_1 PREHOOK: type: QUERY PREHOOK: Input: default@dest_1 PREHOOK: Output: default@dest_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 dest_2 select * from dest_1 POSTHOOK: type: QUERY POSTHOOK: Input: default@dest_1 @@ -262,23 +259,23 @@ STAGE PLANS: Map Operator Tree: TableScan alias: src2 - Statistics: Num rows: 0 Data size: 49 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 9 Data size: 40 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: string) sort order: + Map-reduce partition columns: key (type: string) - Statistics: Num rows: 0 Data size: 49 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 9 Data size: 40 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Map 4 Map Operator Tree: TableScan alias: src3 - Statistics: Num rows: 0 Data size: 49 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 9 Data size: 40 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: string) sort order: + Map-reduce partition columns: key (type: string) - Statistics: Num rows: 0 Data size: 49 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 9 Data size: 40 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Map 5 Map Operator Tree: @@ -1088,23 +1085,23 @@ STAGE PLANS: Map Operator Tree: TableScan alias: src2 - Statistics: Num rows: 0 Data size: 49 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 9 Data size: 40 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: string) sort order: + Map-reduce partition columns: key (type: string) - Statistics: Num rows: 0 Data size: 49 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 9 Data size: 40 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Map 4 Map Operator Tree: TableScan alias: src3 - Statistics: Num rows: 0 Data size: 49 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 9 Data size: 40 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: string) sort order: + Map-reduce partition columns: key (type: string) - Statistics: Num rows: 0 Data size: 49 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 9 Data size: 40 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Map 5 Map Operator Tree: 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_lateral_view.q.out ql/src/test/results/clientpositive/spark/multi_insert_lateral_view.q.out index bca846a..aba1ed8 100644 --- ql/src/test/results/clientpositive/spark/multi_insert_lateral_view.q.out +++ ql/src/test/results/clientpositive/spark/multi_insert_lateral_view.q.out @@ -5,7 +5,6 @@ PREHOOK: type: CREATETABLE_AS_SELECT PREHOOK: Input: default@src PREHOOK: Output: database:default PREHOOK: Output: default@src_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: -- SORT_QUERY_RESULTS create table src_10 as select * from src limit 10 @@ -74,23 +73,23 @@ STAGE PLANS: Map Operator Tree: TableScan alias: src_10 - Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE Lateral View Forward - Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string) outputColumnNames: key - Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE Lateral View Join Operator outputColumnNames: _col0, _col5 - Statistics: Num rows: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col5 (type: double) outputColumnNames: _col0, _col1 - Statistics: Num rows: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -99,41 +98,41 @@ STAGE PLANS: Select Operator expressions: array((key + 1),(key + 2)) (type: array) outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE UDTF Operator - Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE function name: explode Lateral View Join Operator outputColumnNames: _col0, _col5 - Statistics: Num rows: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col5 (type: double) outputColumnNames: _col0, _col1 - Statistics: Num rows: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 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 name: default.src_lv1 Lateral View Forward - Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string) outputColumnNames: key - Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE Lateral View Join Operator outputColumnNames: _col0, _col5 - Statistics: Num rows: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col5 (type: double) outputColumnNames: _col0, _col1 - Statistics: Num rows: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -142,20 +141,20 @@ STAGE PLANS: Select Operator expressions: array((key + 3),(key + 4)) (type: array) outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE UDTF Operator - Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE function name: explode Lateral View Join Operator outputColumnNames: _col0, _col5 - Statistics: Num rows: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col5 (type: double) outputColumnNames: _col0, _col1 - Statistics: Num rows: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -198,8 +197,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src_10 PREHOOK: Output: default@src_lv1 PREHOOK: Output: default@src_lv2 -[Error 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_10 insert overwrite table src_lv1 select key, C lateral view explode(array(key+1, key+2)) A as C insert overwrite table src_lv2 select key, C lateral view explode(array(key+3, key+4)) A as C @@ -307,108 +304,108 @@ STAGE PLANS: Map Operator Tree: TableScan alias: src_10 - Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE Lateral View Forward - Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string) outputColumnNames: key - Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE Lateral View Join Operator outputColumnNames: _col0, _col5 - Statistics: Num rows: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col5 (type: double) outputColumnNames: _col0, _col5 - Statistics: Num rows: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: sum(_col5) keys: _col0 (type: string) mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 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: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: double) Select Operator expressions: array((key + 1),(key + 2)) (type: array) outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE UDTF Operator - Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE function name: explode Lateral View Join Operator outputColumnNames: _col0, _col5 - Statistics: Num rows: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col5 (type: double) outputColumnNames: _col0, _col5 - Statistics: Num rows: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: sum(_col5) keys: _col0 (type: string) mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 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: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: double) Lateral View Forward - Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string) outputColumnNames: key - Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE Lateral View Join Operator outputColumnNames: _col0, _col5 - Statistics: Num rows: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col5 (type: double) outputColumnNames: _col0, _col5 - Statistics: Num rows: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: sum(_col5) keys: _col0 (type: string) mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 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: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: double) Select Operator expressions: array((key + 3),(key + 4)) (type: array) outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE UDTF Operator - Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE function name: explode Lateral View Join Operator outputColumnNames: _col0, _col5 - Statistics: Num rows: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col5 (type: double) outputColumnNames: _col0, _col5 - Statistics: Num rows: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: sum(_col5) keys: _col0 (type: string) mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 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: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: double) Reducer 2 Reduce Operator Tree: @@ -417,14 +414,14 @@ STAGE PLANS: keys: KEY._col0 (type: string) mode: mergepartial outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col1 (type: double) outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -437,14 +434,14 @@ STAGE PLANS: keys: KEY._col0 (type: string) mode: mergepartial outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col1 (type: double) outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -487,8 +484,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src_10 PREHOOK: Output: default@src_lv1 PREHOOK: Output: default@src_lv2 -[Error 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_10 insert overwrite table src_lv1 select key, sum(C) lateral view explode(array(key+1, key+2)) A as C group by key insert overwrite table src_lv2 select key, sum(C) lateral view explode(array(key+3, key+4)) A as C group by key @@ -580,70 +575,70 @@ STAGE PLANS: Map Operator Tree: TableScan alias: src_10 - Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE Lateral View Forward - Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string) outputColumnNames: key - Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE Lateral View Join Operator outputColumnNames: _col0, _col5 - Statistics: Num rows: 0 Data size: 228 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col5 (type: double) outputColumnNames: _col0, _col5 - Statistics: Num rows: 0 Data size: 228 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: sum(_col5) 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: 20 Data size: 208 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: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: double) Select Operator expressions: array((key + 1),(key + 2)) (type: array) outputColumnNames: _col0 - Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE UDTF Operator - Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE function name: explode Lateral View Join Operator outputColumnNames: _col0, _col5 - Statistics: Num rows: 0 Data size: 228 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col5 (type: double) outputColumnNames: _col0, _col5 - Statistics: Num rows: 0 Data size: 228 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: sum(_col5) 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: 20 Data size: 208 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: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: double) Filter Operator predicate: ((key > 200) or (key < 200)) (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 62 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string), value (type: string) outputColumnNames: key, value - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 62 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: string) sort order: + Map-reduce partition columns: key (type: string) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 62 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Reducer 2 Reduce Operator Tree: @@ -652,14 +647,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: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col1 (type: double) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 10 Data size: 104 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: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -668,23 +663,23 @@ STAGE PLANS: Reducer 3 Reduce Operator Tree: Forward - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 62 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (KEY._col0 > 200) (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 2 Data size: 20 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(VALUE._col0) keys: KEY._col0 (type: string) mode: complete outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 10 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: 10 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: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -692,20 +687,20 @@ STAGE PLANS: name: default.src_lv2 Filter Operator predicate: (KEY._col0 < 200) (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 2 Data size: 20 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(VALUE._col0) keys: KEY._col0 (type: string) mode: complete outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 10 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: 10 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: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -763,9 +758,6 @@ PREHOOK: Input: default@src_10 PREHOOK: Output: default@src_lv1 PREHOOK: Output: default@src_lv2 PREHOOK: Output: default@src_lv3 -[Error 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_10 insert overwrite table src_lv1 select key, sum(C) lateral view explode(array(key+1, key+2)) A as C group by key insert overwrite table src_lv2 select key, count(value) where key > 200 group by key @@ -874,120 +866,120 @@ STAGE PLANS: Map Operator Tree: TableScan alias: src_10 - Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE Lateral View Forward - Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string) outputColumnNames: key - Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE Lateral View Join Operator outputColumnNames: _col0, _col5 - Statistics: Num rows: 0 Data size: 228 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col5 (type: double), _col0 (type: string) outputColumnNames: _col5, _col0 - Statistics: Num rows: 0 Data size: 228 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: sum(DISTINCT _col0) keys: _col5 (type: double), _col0 (type: string) mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: double), _col1 (type: string) 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: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: array((key + 1),(key + 2)) (type: array) outputColumnNames: _col0 - Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE UDTF Operator - Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE function name: explode Lateral View Join Operator outputColumnNames: _col0, _col5 - Statistics: Num rows: 0 Data size: 228 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col5 (type: double), _col0 (type: string) outputColumnNames: _col5, _col0 - Statistics: Num rows: 0 Data size: 228 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: sum(DISTINCT _col0) keys: _col5 (type: double), _col0 (type: string) mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: double), _col1 (type: string) 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: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Lateral View Forward - Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string) outputColumnNames: key - Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE Lateral View Join Operator outputColumnNames: _col0, _col5 - Statistics: Num rows: 0 Data size: 228 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col5 (type: double), _col0 (type: string) outputColumnNames: _col5, _col0 - Statistics: Num rows: 0 Data size: 228 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: sum(DISTINCT _col0) keys: _col5 (type: double), _col0 (type: string) mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: double), _col1 (type: string) 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: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: array((key + 3),(key + 4)) (type: array) outputColumnNames: _col0 - Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE UDTF Operator - Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE function name: explode Lateral View Join Operator outputColumnNames: _col0, _col5 - Statistics: Num rows: 0 Data size: 228 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col5 (type: double), _col0 (type: string) outputColumnNames: _col5, _col0 - Statistics: Num rows: 0 Data size: 228 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: sum(DISTINCT _col0) keys: _col5 (type: double), _col0 (type: string) mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: double), _col1 (type: string) 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: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: value (type: string), key (type: string) outputColumnNames: value, key - Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: sum(DISTINCT key) keys: value (type: string), key (type: string) mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 10 Data size: 104 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: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Group By Operator @@ -995,14 +987,14 @@ 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: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: double), _col1 (type: double) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 10 Data size: 104 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: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -1015,14 +1007,14 @@ 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: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: double), _col1 (type: double) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 10 Data size: 104 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: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -1035,14 +1027,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: 5 Data size: 52 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col1 (type: double) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 5 Data size: 52 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: 52 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -1100,9 +1092,6 @@ PREHOOK: Input: default@src_10 PREHOOK: Output: default@src_lv1 PREHOOK: Output: default@src_lv2 PREHOOK: Output: default@src_lv3 -[Error 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_10 insert overwrite table src_lv1 select C, sum(distinct key) lateral view explode(array(key+1, key+2)) A as C group by C insert overwrite table src_lv2 select C, sum(distinct key) lateral view explode(array(key+3, key+4)) A as C group by C @@ -1241,117 +1230,117 @@ STAGE PLANS: Map Operator Tree: TableScan alias: src_10 - Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE Lateral View Forward - Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string) outputColumnNames: key - Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE Lateral View Join Operator outputColumnNames: _col0, _col5 - Statistics: Num rows: 0 Data size: 228 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col5 (type: double) outputColumnNames: _col0, _col5 - Statistics: Num rows: 0 Data size: 228 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: sum(DISTINCT _col5) keys: _col0 (type: string), _col5 (type: double) mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 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) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: array((key + 1),(key + 2)) (type: array) outputColumnNames: _col0 - Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE UDTF Operator - Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE function name: explode Lateral View Join Operator outputColumnNames: _col0, _col5 - Statistics: Num rows: 0 Data size: 228 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col5 (type: double) outputColumnNames: _col0, _col5 - Statistics: Num rows: 0 Data size: 228 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: sum(DISTINCT _col5) keys: _col0 (type: string), _col5 (type: double) mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 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) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Lateral View Forward - Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string) outputColumnNames: key - Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE Lateral View Join Operator outputColumnNames: _col0, _col5 - Statistics: Num rows: 0 Data size: 228 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col5 (type: double) outputColumnNames: _col0, _col5 - Statistics: Num rows: 0 Data size: 228 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: sum(DISTINCT _col5) keys: _col0 (type: string), _col5 (type: double) mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 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) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: array((key + 3),(key + 4)) (type: array) outputColumnNames: _col0 - Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE UDTF Operator - Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE function name: explode Lateral View Join Operator outputColumnNames: _col0, _col5 - Statistics: Num rows: 0 Data size: 228 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col5 (type: double) outputColumnNames: _col0, _col5 - Statistics: Num rows: 0 Data size: 228 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: sum(DISTINCT _col5) keys: _col0 (type: string), _col5 (type: double) mode: hash outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 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) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 20 Data size: 208 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: ((key > 200) or (key < 200)) (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 62 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string), value (type: string) outputColumnNames: key, value - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 62 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: value (type: string), key (type: string) sort order: ++ Map-reduce partition columns: value (type: string) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 62 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Group By Operator @@ -1359,14 +1348,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: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col1 (type: double) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 10 Data size: 104 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: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -1379,14 +1368,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: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col1 (type: double) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 10 Data size: 104 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: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -1395,23 +1384,23 @@ STAGE PLANS: Reducer 4 Reduce Operator Tree: Forward - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 62 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (KEY._col1:0._col0 > 200) (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 2 Data size: 20 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: sum(DISTINCT KEY._col1:0._col0) keys: KEY._col0 (type: string) mode: complete outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col1 (type: double) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 10 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: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -1419,20 +1408,20 @@ STAGE PLANS: name: default.src_lv3 Filter Operator predicate: (KEY._col1:0._col0 < 200) (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 2 Data size: 20 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: sum(DISTINCT KEY._col1:0._col0) keys: KEY._col0 (type: string) mode: complete outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col1 (type: double) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 10 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: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -1505,10 +1494,6 @@ PREHOOK: Output: default@src_lv1 PREHOOK: Output: default@src_lv2 PREHOOK: Output: default@src_lv3 PREHOOK: Output: default@src_lv4 -[Error 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_10 insert overwrite table src_lv1 select key, sum(distinct C) lateral view explode(array(key+1, key+2)) A as C group by key insert overwrite table src_lv2 select key, sum(distinct C) lateral view explode(array(key+3, key+4)) A as C group by key 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/multi_insert_move_tasks_share_dependencies.q.out ql/src/test/results/clientpositive/spark/multi_insert_move_tasks_share_dependencies.q.out index 7e768e4..19f6d86 100644 --- ql/src/test/results/clientpositive/spark/multi_insert_move_tasks_share_dependencies.q.out +++ ql/src/test/results/clientpositive/spark/multi_insert_move_tasks_share_dependencies.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 @@ -3154,8 +3130,6 @@ PREHOOK: Input: default@src #### A masked pattern was here #### 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 @@ -3401,8 +3375,6 @@ PREHOOK: Input: default@src #### A masked pattern was here #### 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 @@ -3648,8 +3620,6 @@ PREHOOK: Input: default@src #### A masked pattern was here #### 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 @@ -3895,8 +3865,6 @@ PREHOOK: Input: default@src #### A masked pattern was here #### 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 diff --git ql/src/test/results/clientpositive/spark/multi_join_union.q.out ql/src/test/results/clientpositive/spark/multi_join_union.q.out index 72e6fb5..3f948a9 100644 --- ql/src/test/results/clientpositive/spark/multi_join_union.q.out +++ ql/src/test/results/clientpositive/spark/multi_join_union.q.out @@ -3,7 +3,6 @@ PREHOOK: type: CREATETABLE_AS_SELECT PREHOOK: Input: default@src PREHOOK: Output: database:default PREHOOK: Output: default@src11 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: CREATE TABLE src11 as SELECT * FROM src POSTHOOK: type: CREATETABLE_AS_SELECT POSTHOOK: Input: default@src @@ -14,7 +13,6 @@ PREHOOK: type: CREATETABLE_AS_SELECT PREHOOK: Input: default@src PREHOOK: Output: database:default PREHOOK: Output: default@src12 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: CREATE TABLE src12 as SELECT * FROM src POSTHOOK: type: CREATETABLE_AS_SELECT POSTHOOK: Input: default@src @@ -25,7 +23,6 @@ PREHOOK: type: CREATETABLE_AS_SELECT PREHOOK: Input: default@src PREHOOK: Output: database:default PREHOOK: Output: default@src13 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: CREATE TABLE src13 as SELECT * FROM src POSTHOOK: type: CREATETABLE_AS_SELECT POSTHOOK: Input: default@src @@ -36,7 +33,6 @@ PREHOOK: type: CREATETABLE_AS_SELECT PREHOOK: Input: default@src PREHOOK: Output: database:default PREHOOK: Output: default@src14 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: CREATE TABLE src14 as SELECT * FROM src POSTHOOK: type: CREATETABLE_AS_SELECT POSTHOOK: Input: default@src @@ -69,15 +65,15 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 29 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 and value is not null) (type: boolean) - Statistics: Num rows: 8 Data size: 1603 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: string) sort order: + Map-reduce partition columns: key (type: string) - Statistics: Num rows: 8 Data size: 1603 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Map 4 Map Operator Tree: @@ -100,15 +96,15 @@ STAGE PLANS: Map Operator Tree: TableScan alias: a - Statistics: Num rows: 29 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: 15 Data size: 3006 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: string) sort order: + Map-reduce partition columns: key (type: string) - Statistics: Num rows: 15 Data size: 3006 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Map 7 Map Operator Tree: @@ -136,12 +132,12 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 {KEY.reducesinkkey0} {VALUE._col0} outputColumnNames: _col0, _col1, _col5, _col6 - Statistics: Num rows: 16 Data size: 3306 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col6 (type: string) sort order: + Map-reduce partition columns: _col6 (type: string) - Statistics: Num rows: 16 Data size: 3306 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string) Reducer 3 Reduce Operator Tree: @@ -152,14 +148,14 @@ STAGE PLANS: 0 {VALUE._col0} {VALUE._col1} {VALUE._col5} {KEY.reducesinkkey0} 1 {VALUE._col0} {KEY.reducesinkkey0} outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11 - Statistics: Num rows: 33 Data size: 6613 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string), _col10 (type: string), _col11 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 33 Data size: 6613 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 33 Data size: 6613 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 550 Data size: 5843 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/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/parquet_join.q.out ql/src/test/results/clientpositive/spark/parquet_join.q.out index d5a8684..da2ce07 100644 --- ql/src/test/results/clientpositive/spark/parquet_join.q.out +++ ql/src/test/results/clientpositive/spark/parquet_join.q.out @@ -30,7 +30,6 @@ PREHOOK: query: insert into table staging select distinct key, value from src or PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@staging -[Error 30017]: Skipping 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 staging select distinct key, value from src order by key limit 2 POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -42,7 +41,6 @@ PREHOOK: type: CREATETABLE_AS_SELECT PREHOOK: Input: default@staging PREHOOK: Output: database:default PREHOOK: Output: default@parquet_jointable1 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: create table parquet_jointable1 stored as parquet as select * from staging POSTHOOK: type: CREATETABLE_AS_SELECT POSTHOOK: Input: default@staging @@ -53,7 +51,6 @@ PREHOOK: type: CREATETABLE_AS_SELECT PREHOOK: Input: default@staging PREHOOK: Output: database:default PREHOOK: Output: default@parquet_jointable2 -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: create table parquet_jointable2 stored as parquet as select key,key+1,concat(value,"value") as myvalue from staging POSTHOOK: type: CREATETABLE_AS_SELECT POSTHOOK: Input: default@staging @@ -82,29 +79,29 @@ STAGE PLANS: Map Operator Tree: TableScan alias: p2 - Statistics: Num rows: 4 Data size: 435 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 2 Data size: 217 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 3 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: 217 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE value expressions: myvalue (type: string) Map 3 Map Operator Tree: TableScan alias: p1 - Statistics: Num rows: 74 Data size: 296 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 4 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 37 Data size: 148 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 2 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: 37 Data size: 148 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 2 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator @@ -114,14 +111,14 @@ STAGE PLANS: 0 1 {VALUE._col1} outputColumnNames: _col7 - Statistics: Num rows: 40 Data size: 162 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 2 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col7 (type: string) outputColumnNames: _col0 - Statistics: Num rows: 40 Data size: 162 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 2 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 40 Data size: 162 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 2 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -176,29 +173,29 @@ STAGE PLANS: Map Operator Tree: TableScan alias: p2 - Statistics: Num rows: 4 Data size: 435 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 2 Data size: 217 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 3 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: 217 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE value expressions: myvalue (type: string) Map 3 Map Operator Tree: TableScan alias: p1 - Statistics: Num rows: 74 Data size: 296 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 4 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 37 Data size: 148 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 2 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: 37 Data size: 148 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 2 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator @@ -208,14 +205,14 @@ STAGE PLANS: 0 1 {VALUE._col1} outputColumnNames: _col7 - Statistics: Num rows: 40 Data size: 162 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 2 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col7 (type: string) outputColumnNames: _col0 - Statistics: Num rows: 40 Data size: 162 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 2 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 40 Data size: 162 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 2 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -255,7 +252,6 @@ PREHOOK: query: insert overwrite table parquet_jointable1_bucketed_sorted select PREHOOK: type: QUERY PREHOOK: Input: default@staging PREHOOK: Output: default@parquet_jointable1_bucketed_sorted -[Error 30017]: Skipping 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 parquet_jointable1_bucketed_sorted select key,concat(value,"value1") as value from staging cluster by key POSTHOOK: type: QUERY POSTHOOK: Input: default@staging @@ -274,7 +270,6 @@ PREHOOK: query: insert overwrite table parquet_jointable2_bucketed_sorted select PREHOOK: type: QUERY PREHOOK: Input: default@staging PREHOOK: Output: default@parquet_jointable2_bucketed_sorted -[Error 30017]: Skipping 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 parquet_jointable2_bucketed_sorted select key,concat(value,"value2-1") as value1,concat(value,"value2-2") as value2 from staging cluster by key POSTHOOK: type: QUERY POSTHOOK: Input: default@staging @@ -301,29 +296,29 @@ STAGE PLANS: Map Operator Tree: TableScan alias: p2 - Statistics: Num rows: 5 Data size: 526 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 3 Data size: 315 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 3 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: 3 Data size: 315 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE value expressions: value2 (type: string) Map 3 Map Operator Tree: TableScan alias: p1 - Statistics: Num rows: 3 Data size: 334 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 4 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 2 Data size: 222 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 2 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: 222 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 2 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Reducer 2 Reduce Operator Tree: @@ -334,14 +329,14 @@ STAGE PLANS: 0 {VALUE._col0} 1 {VALUE._col1} outputColumnNames: _col1, _col7 - Statistics: Num rows: 3 Data size: 346 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 2 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col1 (type: string), _col7 (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 3 Data size: 346 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 2 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 3 Data size: 346 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 2 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/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/ppd_join4.q.out ql/src/test/results/clientpositive/spark/ppd_join4.q.out index 36b5ed5..37faba0 100644 --- ql/src/test/results/clientpositive/spark/ppd_join4.q.out +++ ql/src/test/results/clientpositive/spark/ppd_join4.q.out @@ -23,7 +23,6 @@ select 'a','b' from dual PREHOOK: type: QUERY PREHOOK: Input: default@dual PREHOOK: Output: default@test_tbl -[Error 30017]: Skipping 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 test_tbl select 'a','b' from dual POSTHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/spark/ppd_join5.q.out ql/src/test/results/clientpositive/spark/ppd_join5.q.out index aea1b0a..0886360 100644 --- ql/src/test/results/clientpositive/spark/ppd_join5.q.out +++ ql/src/test/results/clientpositive/spark/ppd_join5.q.out @@ -21,8 +21,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: 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 -[Error 30017]: Skipping 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 t1 select 'a','a' insert into table t2 select 'a',2 diff --git ql/src/test/results/clientpositive/spark/ppd_multi_insert.q.out ql/src/test/results/clientpositive/spark/ppd_multi_insert.q.out index 9c3b060..d98a9ff 100644 --- ql/src/test/results/clientpositive/spark/ppd_multi_insert.q.out +++ ql/src/test/results/clientpositive/spark/ppd_multi_insert.q.out @@ -213,9 +213,6 @@ PREHOOK: Output: default@mi1 PREHOOK: Output: default@mi2 PREHOOK: Output: default@mi3@ds=2008-04-08/hr=12 PREHOOK: Output: target/warehouse/mi4.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 -[Error 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 a JOIN src b ON (a.key = b.key) INSERT OVERWRITE TABLE mi1 SELECT a.* WHERE a.key < 100 INSERT OVERWRITE TABLE mi2 SELECT a.key, a.value WHERE a.key >= 100 and a.key < 200 @@ -1477,9 +1474,6 @@ PREHOOK: Output: default@mi1 PREHOOK: Output: default@mi2 PREHOOK: Output: default@mi3@ds=2008-04-08/hr=12 PREHOOK: Output: target/warehouse/mi4.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 -[Error 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 a JOIN src b ON (a.key = b.key) INSERT OVERWRITE TABLE mi1 SELECT a.* WHERE a.key < 100 INSERT OVERWRITE TABLE mi2 SELECT a.key, a.value WHERE a.key >= 100 and a.key < 200 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/semijoin.q.out ql/src/test/results/clientpositive/spark/semijoin.q.out index 18fc837..4444abb 100644 --- ql/src/test/results/clientpositive/spark/semijoin.q.out +++ ql/src/test/results/clientpositive/spark/semijoin.q.out @@ -3,7 +3,6 @@ PREHOOK: type: CREATETABLE_AS_SELECT PREHOOK: Input: default@src PREHOOK: Output: database:default 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: create table t1 as select cast(key as int) key, value from src where key <= 10 POSTHOOK: type: CREATETABLE_AS_SELECT POSTHOOK: Input: default@src @@ -33,7 +32,6 @@ PREHOOK: type: CREATETABLE_AS_SELECT PREHOOK: Input: default@t1 PREHOOK: Output: database:default 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: create table t2 as select cast(2*key as int) key, value from t1 POSTHOOK: type: CREATETABLE_AS_SELECT POSTHOOK: Input: default@t1 @@ -64,7 +62,6 @@ PREHOOK: Input: default@t1 PREHOOK: Input: default@t2 PREHOOK: Output: database:default 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 POSTHOOK: query: create table t3 as select * from (select * from t1 union all select * from t2) b POSTHOOK: type: CREATETABLE_AS_SELECT POSTHOOK: Input: default@t1 @@ -137,37 +134,37 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 23 Data size: 95 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 84 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 12 Data size: 49 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 45 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int) outputColumnNames: key - Statistics: Num rows: 12 Data size: 49 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 45 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: key (type: int) mode: hash outputColumnNames: _col0 - Statistics: Num rows: 12 Data size: 49 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 45 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: 12 Data size: 49 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 45 Basic stats: COMPLETE Column stats: NONE Map 4 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 0 Data size: 90 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 11 Data size: 79 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: 6 Data size: 43 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: 6 Data size: 43 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Reducer 2 Reduce Operator Tree: @@ -178,24 +175,24 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 outputColumnNames: _col0, _col1 - Statistics: Num rows: 13 Data size: 53 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 47 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 13 Data size: 53 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 47 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int), _col1 (type: string) sort order: ++ - Statistics: Num rows: 13 Data size: 53 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 47 Basic stats: COMPLETE Column stats: NONE Reducer 3 Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 13 Data size: 53 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 47 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 13 Data size: 53 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 47 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -243,37 +240,37 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 22 Data size: 90 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 79 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 11 Data size: 45 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 43 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int) outputColumnNames: key - Statistics: Num rows: 11 Data size: 45 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 43 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: key (type: int) mode: hash outputColumnNames: _col0 - Statistics: Num rows: 11 Data size: 45 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 43 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: 45 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 43 Basic stats: COMPLETE Column stats: NONE Map 4 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 0 Data size: 95 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 11 Data size: 84 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: 6 Data size: 45 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: 6 Data size: 45 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Reducer 2 Reduce Operator Tree: @@ -284,24 +281,24 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 outputColumnNames: _col0, _col1 - Statistics: Num rows: 12 Data size: 49 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 49 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 12 Data size: 49 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 49 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int), _col1 (type: string) sort order: ++ - Statistics: Num rows: 12 Data size: 49 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 49 Basic stats: COMPLETE Column stats: NONE Reducer 3 Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 12 Data size: 49 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 49 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 12 Data size: 49 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 49 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -373,15 +370,15 @@ STAGE PLANS: Map Operator Tree: TableScan alias: a - Statistics: Num rows: 0 Data size: 90 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 11 Data size: 79 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: 6 Data size: 43 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: 6 Data size: 43 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Reducer 2 Reduce Operator Tree: @@ -392,24 +389,24 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 47 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: 6 Data size: 47 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int), _col1 (type: string) sort order: ++ - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 47 Basic stats: COMPLETE Column stats: NONE Reducer 3 Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 47 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: 6 Data size: 47 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -473,15 +470,15 @@ STAGE PLANS: Map Operator Tree: TableScan alias: a - Statistics: Num rows: 0 Data size: 90 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 11 Data size: 79 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: 6 Data size: 43 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: 6 Data size: 43 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Reducer 2 Reduce Operator Tree: @@ -562,37 +559,37 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 0 Data size: 95 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 11 Data size: 84 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: ((value < 'val_10') and key is not null) (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 2 Data size: 15 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int), value (type: string) outputColumnNames: key, value - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 2 Data size: 15 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: key (type: int), value (type: string) mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE 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: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 2 Data size: 15 Basic stats: COMPLETE Column stats: NONE Map 4 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 0 Data size: 90 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 11 Data size: 79 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: 6 Data size: 43 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: 6 Data size: 43 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Reducer 2 Reduce Operator Tree: @@ -603,24 +600,24 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 47 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: 6 Data size: 47 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int), _col1 (type: string) sort order: ++ - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 47 Basic stats: COMPLETE Column stats: NONE Reducer 3 Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 47 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: 6 Data size: 47 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -665,15 +662,15 @@ STAGE PLANS: Map Operator Tree: TableScan alias: a - Statistics: Num rows: 0 Data size: 90 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 11 Data size: 79 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: 6 Data size: 43 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: 6 Data size: 43 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Map 4 Map Operator Tree: @@ -768,37 +765,37 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t2 - Statistics: Num rows: 0 Data size: 95 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 11 Data size: 84 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (((key > 5) and (value <= 'val_20')) and 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: 7 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int), value (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: _col0 (type: int), _col1 (type: string) mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE 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: 0 Data size: 0 Basic stats: NONE 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: 0 Data size: 90 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 11 Data size: 79 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: 6 Data size: 43 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: 6 Data size: 43 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Reducer 2 Reduce Operator Tree: @@ -809,24 +806,24 @@ STAGE PLANS: 0 {VALUE._col0} 1 outputColumnNames: _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 47 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col1 (type: string) outputColumnNames: _col0 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 47 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) sort order: + - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 47 Basic stats: COMPLETE Column stats: NONE Reducer 3 Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey0 (type: string) outputColumnNames: _col0 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 47 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: 6 Data size: 47 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -868,37 +865,37 @@ STAGE PLANS: Map Operator Tree: TableScan alias: t1 - Statistics: Num rows: 22 Data size: 90 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 79 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: ((key > 2) and key is not null) (type: boolean) - Statistics: Num rows: 4 Data size: 16 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: 4 Data size: 16 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: _col0 (type: int) mode: hash outputColumnNames: _col0 - Statistics: Num rows: 4 Data size: 16 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: 4 Data size: 16 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: 0 Data size: 95 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 11 Data size: 84 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: 6 Data size: 45 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: 6 Data size: 45 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Reducer 2 Reduce Operator Tree: @@ -909,24 +906,24 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 outputColumnNames: _col0, _col1 - Statistics: Num rows: 4 Data size: 17 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 49 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 4 Data size: 17 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 49 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int), _col1 (type: string) sort order: ++ - Statistics: Num rows: 4 Data size: 17 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 49 Basic stats: COMPLETE Column stats: NONE Reducer 3 Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 4 Data size: 17 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 49 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 4 Data size: 17 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 49 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -973,24 +970,24 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 22 Data size: 90 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 79 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 11 Data size: 45 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 43 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int) outputColumnNames: key - Statistics: Num rows: 11 Data size: 45 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 43 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: key (type: int) mode: hash outputColumnNames: _col0 - Statistics: Num rows: 11 Data size: 45 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 43 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: 45 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 43 Basic stats: COMPLETE Column stats: NONE Map 4 Map Operator Tree: TableScan @@ -1091,37 +1088,37 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 23 Data size: 95 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 84 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (2 * key) is not null (type: boolean) - Statistics: Num rows: 12 Data size: 49 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 45 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int) outputColumnNames: key - Statistics: Num rows: 12 Data size: 49 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 45 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: key (type: int) mode: hash outputColumnNames: _col0 - Statistics: Num rows: 12 Data size: 49 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 45 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: (2 * _col0) (type: int) sort order: + Map-reduce partition columns: (2 * _col0) (type: int) - Statistics: Num rows: 12 Data size: 49 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 45 Basic stats: COMPLETE Column stats: NONE Map 4 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 0 Data size: 90 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 11 Data size: 79 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: 6 Data size: 43 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: 6 Data size: 43 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Reducer 2 Reduce Operator Tree: @@ -1132,24 +1129,24 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 outputColumnNames: _col0, _col1 - Statistics: Num rows: 13 Data size: 53 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 47 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 13 Data size: 53 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 47 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int), _col1 (type: string) sort order: ++ - Statistics: Num rows: 13 Data size: 53 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 47 Basic stats: COMPLETE Column stats: NONE Reducer 3 Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 13 Data size: 53 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 47 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 13 Data size: 53 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 47 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -1195,15 +1192,15 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 0 Data size: 95 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 11 Data size: 84 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: 6 Data size: 45 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: 6 Data size: 45 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Map 4 Map Operator Tree: @@ -1231,15 +1228,15 @@ STAGE PLANS: Map Operator Tree: TableScan alias: a - Statistics: Num rows: 0 Data size: 90 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 11 Data size: 79 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: 6 Data size: 43 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: 6 Data size: 43 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Reducer 2 Reduce Operator Tree: @@ -1328,24 +1325,24 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 0 Data size: 90 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 11 Data size: 79 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (key is not null and value is not null) (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 21 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int), value (type: string) outputColumnNames: key, value - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 21 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: key (type: int), value (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: 21 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int), _col1 (type: string) sort order: ++ Map-reduce partition columns: _col0 (type: int), _col1 (type: string) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 21 Basic stats: COMPLETE Column stats: NONE Map 4 Map Operator Tree: TableScan @@ -1368,24 +1365,24 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {KEY.reducesinkkey1} 1 outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 203 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 23 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 203 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 23 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int), _col1 (type: string) sort order: ++ - Statistics: Num rows: 1 Data size: 203 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 23 Basic stats: COMPLETE Column stats: NONE Reducer 3 Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 203 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 23 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 203 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 23 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -1441,46 +1438,46 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 22 Data size: 90 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 79 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 11 Data size: 45 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 43 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int) outputColumnNames: key - Statistics: Num rows: 11 Data size: 45 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 43 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: key (type: int) mode: hash outputColumnNames: _col0 - Statistics: Num rows: 11 Data size: 45 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 43 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: 45 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 43 Basic stats: COMPLETE Column stats: NONE Map 4 Map Operator Tree: TableScan alias: c - Statistics: Num rows: 23 Data size: 95 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 84 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 12 Data size: 49 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 45 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int) outputColumnNames: key - Statistics: Num rows: 12 Data size: 49 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 45 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: key (type: int) mode: hash outputColumnNames: _col0 - Statistics: Num rows: 12 Data size: 49 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 45 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: 12 Data size: 49 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 45 Basic stats: COMPLETE Column stats: NONE Map 5 Map Operator Tree: TableScan @@ -1580,31 +1577,31 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 22 Data size: 90 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 79 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: 22 Data size: 90 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 79 Basic stats: COMPLETE Column stats: NONE Map 4 Map Operator Tree: TableScan alias: c - Statistics: Num rows: 23 Data size: 95 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 84 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int) outputColumnNames: key - Statistics: Num rows: 23 Data size: 95 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 84 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: key (type: int) mode: hash outputColumnNames: _col0 - Statistics: Num rows: 23 Data size: 95 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 84 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: 23 Data size: 95 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 84 Basic stats: COMPLETE Column stats: NONE Map 5 Map Operator Tree: TableScan @@ -1723,31 +1720,31 @@ STAGE PLANS: Map Operator Tree: TableScan alias: c - Statistics: Num rows: 23 Data size: 95 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 84 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int) outputColumnNames: key - Statistics: Num rows: 23 Data size: 95 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 84 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: key (type: int) mode: hash outputColumnNames: _col0 - Statistics: Num rows: 23 Data size: 95 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 84 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: 23 Data size: 95 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 84 Basic stats: COMPLETE Column stats: NONE Map 5 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 22 Data size: 90 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 79 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: 22 Data size: 90 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 79 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator @@ -1859,31 +1856,31 @@ STAGE PLANS: Map Operator Tree: TableScan alias: c - Statistics: Num rows: 23 Data size: 95 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 84 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int) outputColumnNames: key - Statistics: Num rows: 23 Data size: 95 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 84 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: key (type: int) mode: hash outputColumnNames: _col0 - Statistics: Num rows: 23 Data size: 95 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 84 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: 23 Data size: 95 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 84 Basic stats: COMPLETE Column stats: NONE Map 5 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 22 Data size: 90 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 79 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: 22 Data size: 90 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 79 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator @@ -1985,31 +1982,31 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 23 Data size: 95 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 84 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int) outputColumnNames: key - Statistics: Num rows: 23 Data size: 95 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 84 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: key (type: int) mode: hash outputColumnNames: _col0 - Statistics: Num rows: 23 Data size: 95 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 84 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: 23 Data size: 95 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 84 Basic stats: COMPLETE Column stats: NONE Map 4 Map Operator Tree: TableScan alias: c - Statistics: Num rows: 22 Data size: 90 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 79 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: 22 Data size: 90 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 79 Basic stats: COMPLETE Column stats: NONE Map 5 Map Operator Tree: TableScan @@ -2121,31 +2118,31 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 23 Data size: 95 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 84 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int) outputColumnNames: key - Statistics: Num rows: 23 Data size: 95 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 84 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: key (type: int) mode: hash outputColumnNames: _col0 - Statistics: Num rows: 23 Data size: 95 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 84 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: 23 Data size: 95 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 84 Basic stats: COMPLETE Column stats: NONE Map 4 Map Operator Tree: TableScan alias: c - Statistics: Num rows: 22 Data size: 90 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 79 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: 22 Data size: 90 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 79 Basic stats: COMPLETE Column stats: NONE Map 5 Map Operator Tree: TableScan @@ -2259,31 +2256,31 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 22 Data size: 90 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 79 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int) outputColumnNames: key - Statistics: Num rows: 22 Data size: 90 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 79 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: key (type: int) mode: hash outputColumnNames: _col0 - Statistics: Num rows: 22 Data size: 90 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 79 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: 22 Data size: 90 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 79 Basic stats: COMPLETE Column stats: NONE Map 4 Map Operator Tree: TableScan alias: c - Statistics: Num rows: 23 Data size: 95 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 84 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: 23 Data size: 95 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 84 Basic stats: COMPLETE Column stats: NONE Map 5 Map Operator Tree: TableScan @@ -2409,34 +2406,34 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 23 Data size: 95 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 11 Data size: 84 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 12 Data size: 49 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 45 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int) outputColumnNames: key - Statistics: Num rows: 12 Data size: 49 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 45 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: key (type: int) mode: hash outputColumnNames: _col0 - Statistics: Num rows: 12 Data size: 49 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 45 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: 12 Data size: 49 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 45 Basic stats: COMPLETE Column stats: NONE Map 5 Map Operator Tree: TableScan alias: c - Statistics: Num rows: 0 Data size: 90 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 11 Data size: 79 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: value (type: string) sort order: + Map-reduce partition columns: value (type: string) - Statistics: Num rows: 0 Data size: 90 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 11 Data size: 79 Basic stats: COMPLETE Column stats: NONE Map 6 Map Operator Tree: TableScan @@ -2460,12 +2457,12 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 outputColumnNames: _col0, _col1 - Statistics: Num rows: 13 Data size: 53 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 49 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col1 (type: string) sort order: + Map-reduce partition columns: _col1 (type: string) - Statistics: Num rows: 13 Data size: 53 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 49 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int) Reducer 3 Reduce Operator Tree: @@ -2476,24 +2473,24 @@ STAGE PLANS: 0 {VALUE._col0} 1 outputColumnNames: _col0 - Statistics: Num rows: 14 Data size: 58 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 12 Data size: 86 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int) outputColumnNames: _col0 - Statistics: Num rows: 14 Data size: 58 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 12 Data size: 86 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: int) sort order: + - Statistics: Num rows: 14 Data size: 58 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 12 Data size: 86 Basic stats: COMPLETE Column stats: NONE Reducer 4 Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey0 (type: int) outputColumnNames: _col0 - Statistics: Num rows: 14 Data size: 58 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 12 Data size: 86 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 14 Data size: 58 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 12 Data size: 86 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -2571,24 +2568,24 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 0 Data size: 95 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 11 Data size: 84 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: value is not null (type: boolean) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 45 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: value (type: string) outputColumnNames: value - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 45 Basic stats: COMPLETE Column stats: NONE Group By Operator keys: value (type: string) mode: hash outputColumnNames: _col0 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 45 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: 45 Basic stats: COMPLETE Column stats: NONE Map 3 Map Operator Tree: TableScan @@ -2612,14 +2609,14 @@ STAGE PLANS: 0 {VALUE._col0} 1 outputColumnNames: _col0 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 49 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int) outputColumnNames: _col0 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 49 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: 6 Data size: 49 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/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/skewjoin_noskew.q.out ql/src/test/results/clientpositive/spark/skewjoin_noskew.q.out index d45cdd3..9cd9990 100644 --- ql/src/test/results/clientpositive/spark/skewjoin_noskew.q.out +++ ql/src/test/results/clientpositive/spark/skewjoin_noskew.q.out @@ -109,7 +109,6 @@ PREHOOK: type: CREATETABLE_AS_SELECT PREHOOK: Input: default@src PREHOOK: Output: database:default PREHOOK: Output: default@noskew -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: create table noskew as select a.* from src a join src b on a.key=b.key order by a.key limit 30 POSTHOOK: type: CREATETABLE_AS_SELECT POSTHOOK: Input: default@src diff --git ql/src/test/results/clientpositive/spark/skewjoinopt10.q.out ql/src/test/results/clientpositive/spark/skewjoinopt10.q.out index 16c1ec1..5ffc5ee 100644 --- ql/src/test/results/clientpositive/spark/skewjoinopt10.q.out +++ ql/src/test/results/clientpositive/spark/skewjoinopt10.q.out @@ -30,7 +30,6 @@ PREHOOK: query: insert overwrite table array_valued_T1 select key, array(value) PREHOOK: type: QUERY PREHOOK: Input: default@t1 PREHOOK: Output: default@array_valued_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: insert overwrite table array_valued_T1 select key, array(value) from T1 POSTHOOK: type: QUERY POSTHOOK: Input: default@t1 @@ -77,15 +76,15 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - 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 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 Reduce Output Operator key expressions: key (type: string) sort order: + Map-reduce partition columns: key (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: value (type: array) Reducer 2 Reduce Operator Tree: @@ -96,26 +95,26 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} 1 {VALUE._col0} outputColumnNames: _col0, _col6 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 13 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col6 (type: array) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 13 Basic stats: COMPLETE Column stats: NONE Lateral View Forward - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 13 Basic stats: COMPLETE Column stats: NONE Select Operator SELECT * : (no compute) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 13 Basic stats: COMPLETE Column stats: NONE Lateral View Join Operator outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 26 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col1 (type: array), _col2 (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 26 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: 6 Data size: 26 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -123,20 +122,20 @@ STAGE PLANS: Select Operator expressions: _col1 (type: array) outputColumnNames: _col0 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 13 Basic stats: COMPLETE Column stats: NONE UDTF Operator - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 13 Basic stats: COMPLETE Column stats: NONE function name: explode Lateral View Join Operator outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 26 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col1 (type: array), _col2 (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 6 Data size: 26 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: 6 Data size: 26 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/skewjoinopt15.q.out ql/src/test/results/clientpositive/spark/skewjoinopt15.q.out index e197185..ce5a9b2 100644 --- ql/src/test/results/clientpositive/spark/skewjoinopt15.q.out +++ ql/src/test/results/clientpositive/spark/skewjoinopt15.q.out @@ -28,7 +28,6 @@ PREHOOK: query: INSERT OVERWRITE TABLE T1 SELECT key, val FROM tmpT1 PREHOOK: type: QUERY PREHOOK: Input: default@tmpt1 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: INSERT OVERWRITE TABLE T1 SELECT key, val FROM tmpT1 POSTHOOK: type: QUERY POSTHOOK: Input: default@tmpt1 @@ -63,7 +62,6 @@ PREHOOK: query: INSERT OVERWRITE TABLE T2 SELECT key, val FROM tmpT2 PREHOOK: type: QUERY PREHOOK: Input: default@tmpt2 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: INSERT OVERWRITE TABLE T2 SELECT key, val FROM tmpT2 POSTHOOK: type: QUERY POSTHOOK: Input: default@tmpt2 @@ -103,29 +101,29 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - 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 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 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: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE value expressions: val (type: string) Map 3 Map Operator Tree: TableScan alias: a - 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 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 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: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE value expressions: val (type: string) Reducer 2 Reduce Operator Tree: @@ -136,14 +134,14 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 {KEY.reducesinkkey0} {VALUE._col0} outputColumnNames: _col0, _col1, _col5, _col6 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 13 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: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 13 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: 3 Data size: 13 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -198,23 +196,23 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 value expressions: val (type: string) Map 3 Map Operator Tree: TableScan alias: a - 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 value expressions: val (type: string) Reducer 2 Reduce Operator Tree: @@ -225,14 +223,14 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 {KEY.reducesinkkey0} {VALUE._col0} outputColumnNames: _col0, _col1, _col5, _col6 - Statistics: Num rows: 0 Data size: 33 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 26 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: 0 Data size: 33 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 26 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 0 Data size: 33 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 6 Data size: 26 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -290,28 +288,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 7 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 4 Data size: 17 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 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: 4 Data size: 17 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Map 4 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 7 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 4 Data size: 17 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 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: 4 Data size: 17 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator @@ -320,9 +318,9 @@ STAGE PLANS: condition expressions: 0 1 - Statistics: Num rows: 4 Data size: 18 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 13 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 4 Data size: 18 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 13 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) mode: hash @@ -390,22 +388,22 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 7 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 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: 7 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Map 4 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 7 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 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: 7 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator @@ -414,9 +412,9 @@ STAGE PLANS: condition expressions: 0 1 - Statistics: Num rows: 7 Data size: 33 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 26 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 7 Data size: 33 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6 Data size: 26 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) mode: hash diff --git ql/src/test/results/clientpositive/spark/skewjoinopt18.q.out ql/src/test/results/clientpositive/spark/skewjoinopt18.q.out index f615dfa..3f237ec 100644 --- ql/src/test/results/clientpositive/spark/skewjoinopt18.q.out +++ ql/src/test/results/clientpositive/spark/skewjoinopt18.q.out @@ -28,7 +28,6 @@ PREHOOK: query: INSERT OVERWRITE TABLE T1 SELECT key, val FROM tmpT1 PREHOOK: type: QUERY PREHOOK: Input: default@tmpt1 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: INSERT OVERWRITE TABLE T1 SELECT key, val FROM tmpT1 POSTHOOK: type: QUERY POSTHOOK: Input: default@tmpt1 @@ -104,15 +103,15 @@ STAGE PLANS: Map Operator Tree: TableScan alias: a - 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: UDFToDouble(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 Reduce Output Operator key expressions: UDFToDouble(key) (type: double) sort order: + Map-reduce partition columns: UDFToDouble(key) (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: key (type: int), val (type: string) Reducer 2 Reduce Operator Tree: @@ -123,14 +122,14 @@ STAGE PLANS: 0 {VALUE._col0} {VALUE._col1} 1 {VALUE._col0} {VALUE._col1} outputColumnNames: _col0, _col1, _col5, _col6 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 13 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col5 (type: string), _col6 (type: string) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 3 Data size: 13 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: 3 Data size: 13 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/smb_mapjoin9.q.out ql/src/test/results/clientpositive/spark/smb_mapjoin9.q.out index 43ea4cd..6c6de62 100644 --- ql/src/test/results/clientpositive/spark/smb_mapjoin9.q.out +++ ql/src/test/results/clientpositive/spark/smb_mapjoin9.q.out @@ -342,7 +342,6 @@ PREHOOK: query: insert overwrite table hive_test_smb_bucket1 partition (ds='2010 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@hive_test_smb_bucket1@ds=2010-10-15 -[Error 30017]: Skipping 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 hive_test_smb_bucket1 partition (ds='2010-10-15') select key, value from src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -353,7 +352,6 @@ PREHOOK: query: insert overwrite table hive_test_smb_bucket2 partition (ds='2010 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@hive_test_smb_bucket2@ds=2010-10-15 -[Error 30017]: Skipping 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 hive_test_smb_bucket2 partition (ds='2010-10-15') select key, value from src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -392,29 +390,29 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - 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: 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 value expressions: value (type: string), ds (type: string) Map 3 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 @@ -424,14 +422,14 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} 1 {KEY.reducesinkkey0} {VALUE._col0} {VALUE._col1} outputColumnNames: _col0, _col6, _col7, _col8 - Statistics: Num rows: 799 Data size: 3198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col6 (type: int), _col7 (type: string), _col8 (type: string), _col0 (type: int) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 799 Data size: 3198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 799 Data size: 3198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -471,7 +469,6 @@ PREHOOK: Input: default@hive_test_smb_bucket2 PREHOOK: Input: default@hive_test_smb_bucket2@ds=2010-10-15 PREHOOK: Output: database:default PREHOOK: Output: default@smb_mapjoin9_results -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: create table smb_mapjoin9_results as SELECT /* + MAPJOIN(b) */ b.key as k1, b.value, b.ds, a.key as k2 FROM hive_test_smb_bucket1 a JOIN 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_14.q.out ql/src/test/results/clientpositive/spark/smb_mapjoin_14.q.out index cad4063..8cc739e 100644 --- ql/src/test/results/clientpositive/spark/smb_mapjoin_14.q.out +++ ql/src/test/results/clientpositive/spark/smb_mapjoin_14.q.out @@ -19,7 +19,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 @@ -32,7 +31,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 @@ -68,28 +66,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 @@ -98,9 +96,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 @@ -187,28 +185,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 5 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 @@ -218,22 +216,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: @@ -242,25 +240,25 @@ 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: + - 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: Select Operator expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (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 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 @@ -337,28 +335,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 @@ -368,22 +366,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: @@ -392,9 +390,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 @@ -488,36 +486,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 @@ -526,9 +524,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 @@ -626,32 +624,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 @@ -660,9 +658,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 @@ -784,36 +782,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 @@ -822,9 +820,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 @@ -936,36 +934,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 @@ -974,9 +972,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 @@ -1064,36 +1062,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 @@ -1102,9 +1100,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 @@ -1188,32 +1186,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 @@ -1222,9 +1220,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 @@ -1304,32 +1302,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 @@ -1338,9 +1336,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 @@ -1430,53 +1428,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 @@ -1487,9 +1485,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 @@ -1595,32 +1593,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 @@ -1629,9 +1627,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 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/smb_mapjoin_17.q.out ql/src/test/results/clientpositive/spark/smb_mapjoin_17.q.out index 482268c..1ec11d8 100644 --- ql/src/test/results/clientpositive/spark/smb_mapjoin_17.q.out +++ ql/src/test/results/clientpositive/spark/smb_mapjoin_17.q.out @@ -69,7 +69,6 @@ SELECT * FROM src WHERE key < 10 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@test_table1 -[Error 30017]: Skipping 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_table1 SELECT * FROM src WHERE key < 10 POSTHOOK: type: QUERY @@ -82,7 +81,6 @@ SELECT * FROM src WHERE key < 10 PREHOOK: type: QUERY PREHOOK: Input: default@src 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 POSTHOOK: query: INSERT OVERWRITE TABLE test_table2 SELECT * FROM src WHERE key < 10 POSTHOOK: type: QUERY @@ -95,7 +93,6 @@ SELECT * FROM src WHERE key < 10 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@test_table3 -[Error 30017]: Skipping 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_table3 SELECT * FROM src WHERE key < 10 POSTHOOK: type: QUERY @@ -108,7 +105,6 @@ SELECT * FROM src WHERE key < 10 PREHOOK: type: QUERY PREHOOK: Input: default@src 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 POSTHOOK: query: INSERT OVERWRITE TABLE test_table4 SELECT * FROM src WHERE key < 10 POSTHOOK: type: QUERY @@ -121,7 +117,6 @@ SELECT * FROM src WHERE key < 10 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@test_table5 -[Error 30017]: Skipping 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_table5 SELECT * FROM src WHERE key < 10 POSTHOOK: type: QUERY @@ -134,7 +129,6 @@ SELECT * FROM src WHERE key < 10 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@test_table6 -[Error 30017]: Skipping 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_table6 SELECT * FROM src WHERE key < 10 POSTHOOK: type: QUERY @@ -147,7 +141,6 @@ SELECT * FROM src WHERE key < 10 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@test_table7 -[Error 30017]: Skipping 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_table7 SELECT * FROM src WHERE key < 10 POSTHOOK: type: QUERY @@ -160,7 +153,6 @@ SELECT * FROM src WHERE key < 10 PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@test_table8 -[Error 30017]: Skipping 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_table8 SELECT * FROM src WHERE key < 10 POSTHOOK: type: QUERY @@ -204,93 +196,93 @@ STAGE PLANS: Map Operator Tree: TableScan alias: f - 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: g - 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: d - 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: e - 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 7 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 8 Map Operator Tree: TableScan alias: c - 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: 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 @@ -309,9 +301,9 @@ STAGE PLANS: 4 5 6 - Statistics: Num rows: 66 Data size: 264 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 33 Data size: 231 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 66 Data size: 264 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 33 Data size: 231 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash @@ -417,72 +409,72 @@ STAGE PLANS: Map Operator Tree: TableScan alias: f - 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 Map 4 Map Operator Tree: TableScan alias: g - 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 Map 5 Map Operator Tree: TableScan alias: d - 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 Map 6 Map Operator Tree: TableScan alias: e - 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 Map 7 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 Map 8 Map Operator Tree: TableScan alias: c - 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 Map 9 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 Reducer 2 Reduce Operator Tree: Join Operator @@ -501,9 +493,9 @@ STAGE PLANS: 4 5 6 - Statistics: Num rows: 132 Data size: 528 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 66 Data size: 462 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 132 Data size: 528 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 66 Data size: 462 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash @@ -607,82 +599,82 @@ STAGE PLANS: Map Operator Tree: TableScan alias: f - 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 Map 10 Map Operator Tree: TableScan alias: h - 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 Map 4 Map Operator Tree: TableScan alias: g - 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 Map 5 Map Operator Tree: TableScan alias: d - 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 Map 6 Map Operator Tree: TableScan alias: e - 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 Map 7 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 Map 8 Map Operator Tree: TableScan alias: c - 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 Map 9 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 Reducer 2 Reduce Operator Tree: Join Operator @@ -703,9 +695,9 @@ STAGE PLANS: 5 6 7 - Statistics: Num rows: 154 Data size: 616 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 77 Data size: 539 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 154 Data size: 616 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 77 Data size: 539 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash @@ -841,202 +833,202 @@ STAGE PLANS: Map Operator Tree: TableScan alias: f - 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 Map 10 Map Operator Tree: TableScan alias: n - 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 Map 11 Map Operator Tree: TableScan alias: o - 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 Map 12 Map Operator Tree: TableScan alias: l - 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 Map 13 Map Operator Tree: TableScan alias: m - 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 Map 14 Map Operator Tree: TableScan alias: j - 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 Map 15 Map Operator Tree: TableScan alias: k - 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 Map 16 Map Operator Tree: TableScan alias: h - 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 Map 17 Map Operator Tree: TableScan alias: i - 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 Map 18 Map Operator Tree: TableScan alias: t - 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 Map 19 Map Operator Tree: TableScan alias: s - 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 Map 20 Map Operator Tree: TableScan alias: r - 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 Map 21 Map Operator Tree: TableScan alias: q - 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 Map 22 Map Operator Tree: TableScan alias: p - 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 Map 4 Map Operator Tree: TableScan alias: g - 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 Map 5 Map Operator Tree: TableScan alias: d - 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 Map 6 Map Operator Tree: TableScan alias: e - 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 Map 7 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 Map 8 Map Operator Tree: TableScan alias: c - 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 Map 9 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 value expressions: value (type: string) Reducer 2 Reduce Operator Tree: @@ -1075,12 +1067,12 @@ STAGE PLANS: 14 15 outputColumnNames: _col0, _col1 - Statistics: Num rows: 330 Data size: 1320 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 165 Data size: 1155 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: 330 Data size: 1320 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 165 Data size: 1155 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string) Reducer 3 Reduce Operator Tree: @@ -1097,14 +1089,14 @@ STAGE PLANS: 3 4 outputColumnNames: _col0, _col1 - Statistics: Num rows: 1452 Data size: 5808 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 726 Data size: 5082 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 1452 Data size: 5808 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 726 Data size: 5082 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 1452 Data size: 5808 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 726 Data size: 5082 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/smb_mapjoin_18.q.out ql/src/test/results/clientpositive/spark/smb_mapjoin_18.q.out index e06a6ff..7108ca4 100644 --- ql/src/test/results/clientpositive/spark/smb_mapjoin_18.q.out +++ ql/src/test/results/clientpositive/spark/smb_mapjoin_18.q.out @@ -25,7 +25,6 @@ INSERT OVERWRITE TABLE test_table1 PARTITION (ds = '1') SELECT * PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@test_table1@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: FROM src INSERT OVERWRITE TABLE test_table1 PARTITION (ds = '1') SELECT * POSTHOOK: type: QUERY @@ -60,14 +59,14 @@ 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 Select Operator expressions: key (type: int), value (type: string) outputColumnNames: _col0, _col1 - 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 File Output Operator compressed: false - 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 table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -98,7 +97,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@test_table1 PREHOOK: Input: default@test_table1@ds=1 PREHOOK: Output: default@test_table2@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 test_table2 PARTITION (ds = '1') SELECT a.key, a.value FROM test_table1 a WHERE a.ds = '1' POSTHOOK: type: QUERY @@ -246,17 +244,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 Filter Operator predicate: (key = 238) (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 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 @@ -287,7 +285,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@test_table1 PREHOOK: Input: default@test_table1@ds=1 PREHOOK: Output: default@test_table2@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 test_table2 PARTITION (ds = '2') SELECT a.key, a.value FROM test_table1 a WHERE a.ds = '1' and a.key = 238 POSTHOOK: type: QUERY @@ -374,14 +371,14 @@ STAGE PLANS: Map Operator Tree: TableScan alias: a - Statistics: Num rows: 0 Data size: 24 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 2 Data size: 22 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int), value (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 0 Data size: 24 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 2 Data size: 22 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 0 Data size: 24 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 2 Data size: 22 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -412,7 +409,7 @@ PREHOOK: type: QUERY PREHOOK: Input: default@test_table2 PREHOOK: Input: default@test_table2@ds=2 PREHOOK: Output: default@test_table2@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 +Status: Failed POSTHOOK: query: INSERT OVERWRITE TABLE test_table2 PARTITION (ds = '2') SELECT a.key, a.value FROM test_table2 a WHERE a.ds = '2' POSTHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/spark/smb_mapjoin_19.q.out ql/src/test/results/clientpositive/spark/smb_mapjoin_19.q.out index 868d891..fda9db6 100644 --- ql/src/test/results/clientpositive/spark/smb_mapjoin_19.q.out +++ ql/src/test/results/clientpositive/spark/smb_mapjoin_19.q.out @@ -25,7 +25,6 @@ INSERT OVERWRITE TABLE test_table1 PARTITION (ds = '1') SELECT * PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@test_table1@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: FROM src INSERT OVERWRITE TABLE test_table1 PARTITION (ds = '1') SELECT * POSTHOOK: type: QUERY @@ -60,14 +59,14 @@ 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 Select Operator expressions: key (type: int), value (type: string) outputColumnNames: _col0, _col1 - 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 File Output Operator compressed: false - 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 table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -98,7 +97,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@test_table1 PREHOOK: Input: default@test_table1@ds=1 PREHOOK: Output: default@test_table2@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 test_table2 PARTITION (ds = '1') SELECT a.key, a.value FROM test_table1 a WHERE a.ds = '1' POSTHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/spark/smb_mapjoin_20.q.out ql/src/test/results/clientpositive/spark/smb_mapjoin_20.q.out index 292f596..42bab98 100644 --- ql/src/test/results/clientpositive/spark/smb_mapjoin_20.q.out +++ ql/src/test/results/clientpositive/spark/smb_mapjoin_20.q.out @@ -25,7 +25,6 @@ INSERT OVERWRITE TABLE test_table1 PARTITION (ds = '1') SELECT * PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@test_table1@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: FROM src INSERT OVERWRITE TABLE test_table1 PARTITION (ds = '1') SELECT * POSTHOOK: type: QUERY @@ -62,24 +61,24 @@ 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 Select Operator expressions: key (type: int), value (type: string), value (type: string) outputColumnNames: _col0, _col1, _col2 - 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 Reduce Output Operator key expressions: UDFToString(_col0) (type: string) sort order: + Map-reduce partition columns: UDFToString(_col0) (type: string) - 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 value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string) Reducer 2 Reduce Operator Tree: Extract - 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 File Output Operator compressed: false - 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 table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -110,7 +109,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@test_table1 PREHOOK: Input: default@test_table1@ds=1 PREHOOK: Output: default@test_table2@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 test_table2 PARTITION (ds = '1') SELECT a.key, a.value, a.value FROM test_table1 a WHERE a.ds = '1' POSTHOOK: type: QUERY @@ -190,14 +188,14 @@ 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 Select Operator expressions: value (type: string), key (type: int), value (type: string) outputColumnNames: _col0, _col1, _col2 - 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 File Output Operator compressed: false - 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 table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -228,7 +226,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@test_table1 PREHOOK: Input: default@test_table1@ds=1 PREHOOK: Output: default@test_table3@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 test_table3 PARTITION (ds = '1') SELECT a.value, a.key, a.value FROM test_table1 a WHERE a.ds = '1' POSTHOOK: type: QUERY @@ -322,24 +319,24 @@ 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 Select Operator expressions: (key + key) (type: int), value (type: string), value (type: string) outputColumnNames: _col0, _col1, _col2 - 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 Reduce Output Operator key expressions: UDFToString(_col0) (type: string) sort order: + Map-reduce partition columns: UDFToString(_col0) (type: string) - 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 value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string) Reducer 2 Reduce Operator Tree: Extract - 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 File Output Operator compressed: false - 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 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/smb_mapjoin_21.q.out ql/src/test/results/clientpositive/spark/smb_mapjoin_21.q.out index 8bc5dd6..48fed8d 100644 --- ql/src/test/results/clientpositive/spark/smb_mapjoin_21.q.out +++ ql/src/test/results/clientpositive/spark/smb_mapjoin_21.q.out @@ -25,7 +25,6 @@ INSERT OVERWRITE TABLE test_table1 PARTITION (ds = '1') SELECT * PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@test_table1@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: FROM src INSERT OVERWRITE TABLE test_table1 PARTITION (ds = '1') SELECT * POSTHOOK: type: QUERY @@ -60,14 +59,14 @@ 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 Select Operator expressions: key (type: int), value (type: string) outputColumnNames: _col0, _col1 - 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 File Output Operator compressed: false - 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 table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -139,24 +138,24 @@ 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 Select Operator expressions: key (type: int), value (type: string) outputColumnNames: _col0, _col1 - 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 Reduce Output Operator key expressions: _col0 (type: int) sort order: - Map-reduce partition columns: _col0 (type: int) - 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 value expressions: _col0 (type: int), _col1 (type: string) Reducer 2 Reduce Operator Tree: Extract - 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 File Output Operator compressed: false - 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 table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -228,24 +227,24 @@ 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 Select Operator expressions: key (type: int), value (type: string) outputColumnNames: _col0, _col1 - 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 Reduce Output Operator key expressions: _col0 (type: int), _col1 (type: string) sort order: ++ Map-reduce partition columns: _col0 (type: int) - 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 value expressions: _col0 (type: int), _col1 (type: string) Reducer 2 Reduce Operator Tree: Extract - 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 File Output Operator compressed: false - 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 table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -317,24 +316,24 @@ 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 Select Operator expressions: key (type: int), value (type: string) outputColumnNames: _col0, _col1 - 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 Reduce Output Operator key expressions: _col1 (type: string) sort order: + Map-reduce partition columns: _col0 (type: int) - 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 value expressions: _col0 (type: int), _col1 (type: string) Reducer 2 Reduce Operator Tree: Extract - 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 File Output Operator compressed: false - 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 table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -406,24 +405,24 @@ 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 Select Operator expressions: key (type: int), value (type: string) outputColumnNames: _col0, _col1 - 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 Reduce Output Operator key expressions: _col0 (type: int) sort order: + Map-reduce partition columns: _col0 (type: int) - 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 value expressions: _col0 (type: int), _col1 (type: string) Reducer 2 Reduce Operator Tree: Extract - 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 File Output Operator compressed: false - 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 table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -495,23 +494,23 @@ 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 Select Operator expressions: key (type: int), value (type: string) outputColumnNames: _col0, _col1 - 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 Reduce Output Operator sort order: Map-reduce partition columns: _col0 (type: int) - 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 value expressions: _col0 (type: int), _col1 (type: string) Reducer 2 Reduce Operator Tree: Extract - 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 File Output Operator compressed: false - 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 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/smb_mapjoin_22.q.out ql/src/test/results/clientpositive/spark/smb_mapjoin_22.q.out index 8c0de58..b9a8a2a 100644 --- ql/src/test/results/clientpositive/spark/smb_mapjoin_22.q.out +++ ql/src/test/results/clientpositive/spark/smb_mapjoin_22.q.out @@ -25,7 +25,6 @@ INSERT OVERWRITE TABLE test_table1 SELECT * PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@test_table1 -[Error 30017]: Skipping 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 * POSTHOOK: type: QUERY @@ -58,14 +57,14 @@ STAGE PLANS: Map Operator Tree: TableScan alias: test_table1 - 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 Select Operator expressions: key (type: int), value (type: string) outputColumnNames: _col0, _col1 - 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 File Output Operator compressed: false - 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 table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -93,7 +92,6 @@ SELECT * FROM test_table1 PREHOOK: type: QUERY PREHOOK: Input: 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 POSTHOOK: query: INSERT OVERWRITE TABLE test_table2 SELECT * FROM test_table1 POSTHOOK: type: QUERY @@ -178,7 +176,6 @@ INSERT OVERWRITE TABLE test_table1 SELECT * PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@test_table1 -[Error 30017]: Skipping 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 * POSTHOOK: type: QUERY @@ -211,14 +208,14 @@ STAGE PLANS: Map Operator Tree: TableScan alias: test_table1 - 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 Select Operator expressions: key (type: int), value (type: string) outputColumnNames: _col0, _col1 - 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 File Output Operator compressed: false - 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 table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -246,7 +243,6 @@ SELECT * FROM test_table1 PREHOOK: type: QUERY PREHOOK: Input: 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 POSTHOOK: query: INSERT OVERWRITE TABLE test_table2 SELECT * FROM test_table1 POSTHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/spark/smb_mapjoin_6.q.out ql/src/test/results/clientpositive/spark/smb_mapjoin_6.q.out index 1836f12..ecd1a8c 100644 --- ql/src/test/results/clientpositive/spark/smb_mapjoin_6.q.out +++ ql/src/test/results/clientpositive/spark/smb_mapjoin_6.q.out @@ -35,7 +35,6 @@ select * from src PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@smb_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 smb_bucket4_1 select * from src POSTHOOK: type: QUERY @@ -48,7 +47,6 @@ select * from src PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@smb_bucket4_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 smb_bucket4_2 select * from src POSTHOOK: type: QUERY @@ -81,29 +79,29 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 48 Data size: 5008 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 500 Data size: 4812 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 24 Data size: 2504 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2406 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: 24 Data size: 2504 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2406 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Map 3 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 48 Data size: 5008 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 500 Data size: 4812 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 24 Data size: 2504 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2406 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: 24 Data size: 2504 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2406 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Reducer 2 Reduce Operator Tree: @@ -114,14 +112,14 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 {KEY.reducesinkkey0} {VALUE._col0} outputColumnNames: _col0, _col1, _col5, _col6 - Statistics: Num rows: 26 Data size: 2754 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2646 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: 26 Data size: 2754 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2646 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 26 Data size: 2754 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2646 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -150,7 +148,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@smb_bucket4_1 PREHOOK: Input: default@smb_bucket4_2 PREHOOK: Output: default@smb_join_results -[Error 30017]: Skipping 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 smb_join_results select /*+mapjoin(a)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key POSTHOOK: type: QUERY @@ -1202,7 +1199,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@smb_bucket4_1 PREHOOK: Input: default@smb_bucket4_2 PREHOOK: Output: default@normal_join_results -[Error 30017]: Skipping 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 normal_join_results select * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key POSTHOOK: type: QUERY POSTHOOK: Input: default@smb_bucket4_1 @@ -1255,29 +1251,29 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 48 Data size: 5008 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 500 Data size: 4812 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 24 Data size: 2504 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2406 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: 24 Data size: 2504 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2406 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Map 3 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 48 Data size: 5008 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 500 Data size: 4812 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 24 Data size: 2504 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2406 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: 24 Data size: 2504 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2406 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Reducer 2 Reduce Operator Tree: @@ -1288,14 +1284,14 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 {KEY.reducesinkkey0} {VALUE._col0} outputColumnNames: _col0, _col1, _col5, _col6 - Statistics: Num rows: 26 Data size: 2754 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2646 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: 26 Data size: 2754 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2646 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 26 Data size: 2754 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2646 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -1324,7 +1320,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@smb_bucket4_1 PREHOOK: Input: default@smb_bucket4_2 PREHOOK: Output: default@smb_join_results -[Error 30017]: Skipping 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 smb_join_results select /*+mapjoin(b)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key POSTHOOK: type: QUERY @@ -1341,7 +1336,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@smb_bucket4_1 PREHOOK: Input: default@smb_bucket4_2 PREHOOK: Output: default@smb_join_results -[Error 30017]: Skipping 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 smb_join_results select /*+mapjoin(a)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key POSTHOOK: type: QUERY @@ -2393,7 +2387,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@smb_bucket4_1 PREHOOK: Input: default@smb_bucket4_2 PREHOOK: Output: default@normal_join_results -[Error 30017]: Skipping 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 normal_join_results select * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key POSTHOOK: type: QUERY POSTHOOK: Input: default@smb_bucket4_1 @@ -2446,29 +2439,29 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 48 Data size: 5008 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 500 Data size: 4812 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (key is not null and (key > 1000)) (type: boolean) - Statistics: Num rows: 8 Data size: 834 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 83 Data size: 798 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: 8 Data size: 834 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 83 Data size: 798 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Map 3 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 48 Data size: 5008 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 500 Data size: 4812 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (key is not null and (key > 1000)) (type: boolean) - Statistics: Num rows: 8 Data size: 834 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 83 Data size: 798 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: 8 Data size: 834 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 83 Data size: 798 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Reducer 2 Reduce Operator Tree: @@ -2479,14 +2472,14 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 {KEY.reducesinkkey0} {VALUE._col0} outputColumnNames: _col0, _col1, _col5, _col6 - Statistics: Num rows: 8 Data size: 917 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 91 Data size: 877 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: 8 Data size: 917 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 91 Data size: 877 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 8 Data size: 917 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 91 Data size: 877 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -2515,7 +2508,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@smb_bucket4_1 PREHOOK: Input: default@smb_bucket4_2 PREHOOK: Output: default@smb_join_results -[Error 30017]: Skipping 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 smb_join_results select /*+mapjoin(a)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key where a.key>1000 POSTHOOK: type: QUERY @@ -2551,29 +2543,29 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 48 Data size: 5008 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 500 Data size: 4812 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (key is not null and (key > 1000)) (type: boolean) - Statistics: Num rows: 8 Data size: 834 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 83 Data size: 798 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: 8 Data size: 834 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 83 Data size: 798 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Map 3 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 48 Data size: 5008 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 500 Data size: 4812 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (key is not null and (key > 1000)) (type: boolean) - Statistics: Num rows: 8 Data size: 834 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 83 Data size: 798 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: 8 Data size: 834 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 83 Data size: 798 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Reducer 2 Reduce Operator Tree: @@ -2584,14 +2576,14 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 {KEY.reducesinkkey0} {VALUE._col0} outputColumnNames: _col0, _col1, _col5, _col6 - Statistics: Num rows: 8 Data size: 917 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 91 Data size: 877 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: 8 Data size: 917 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 91 Data size: 877 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 8 Data size: 917 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 91 Data size: 877 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -2620,7 +2612,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@smb_bucket4_1 PREHOOK: Input: default@smb_bucket4_2 PREHOOK: Output: default@smb_join_results -[Error 30017]: Skipping 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 smb_join_results select /*+mapjoin(b)*/ * from smb_bucket4_1 a join smb_bucket4_2 b on a.key = b.key where a.key>1000 POSTHOOK: type: QUERY @@ -2652,43 +2643,43 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 48 Data size: 5008 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 500 Data size: 4812 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (key is not null and (key > 1000)) (type: boolean) - Statistics: Num rows: 8 Data size: 834 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 83 Data size: 798 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: 8 Data size: 834 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 83 Data size: 798 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Map 3 Map Operator Tree: TableScan alias: c - Statistics: Num rows: 48 Data size: 5008 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 500 Data size: 4812 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (key is not null and (key > 1000)) (type: boolean) - Statistics: Num rows: 8 Data size: 834 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 83 Data size: 798 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: 8 Data size: 834 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 83 Data size: 798 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Map 4 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 48 Data size: 5008 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 500 Data size: 4812 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (key is not null and (key > 1000)) (type: boolean) - Statistics: Num rows: 8 Data size: 834 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 83 Data size: 798 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: 8 Data size: 834 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 83 Data size: 798 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Reducer 2 Reduce Operator Tree: @@ -2701,14 +2692,14 @@ STAGE PLANS: 1 {KEY.reducesinkkey0} {VALUE._col0} 2 {KEY.reducesinkkey0} {VALUE._col0} outputColumnNames: _col0, _col1, _col5, _col6, _col10, _col11 - Statistics: Num rows: 17 Data size: 1834 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 182 Data size: 1755 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: string), _col5 (type: int), _col6 (type: string), _col10 (type: int), _col11 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 17 Data size: 1834 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 182 Data size: 1755 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 17 Data size: 1834 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 182 Data size: 1755 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/smb_mapjoin_7.q.out ql/src/test/results/clientpositive/spark/smb_mapjoin_7.q.out index 4ba1585..1e003f3 100644 --- ql/src/test/results/clientpositive/spark/smb_mapjoin_7.q.out +++ ql/src/test/results/clientpositive/spark/smb_mapjoin_7.q.out @@ -59,7 +59,6 @@ select * from src PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@smb_bucket4_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 smb_bucket4_2 select * from src POSTHOOK: type: QUERY @@ -73,7 +72,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@smb_bucket4_1 PREHOOK: Input: default@smb_bucket4_2 PREHOOK: Output: default@smb_join_results_empty_bigtable -[Error 30017]: Skipping 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 smb_join_results_empty_bigtable select /*+mapjoin(b)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key POSTHOOK: type: QUERY @@ -90,7 +88,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@smb_bucket4_1 PREHOOK: Input: default@smb_bucket4_2 PREHOOK: Output: default@smb_join_results_empty_bigtable -[Error 30017]: Skipping 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 smb_join_results_empty_bigtable select /*+mapjoin(b)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key POSTHOOK: type: QUERY @@ -634,12 +631,12 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 value expressions: value (type: string) Map 3 Map Operator Tree: @@ -661,14 +658,14 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} {VALUE._col0} 1 {KEY.reducesinkkey0} {VALUE._col0} outputColumnNames: _col0, _col1, _col5, _col6 - Statistics: Num rows: 60 Data size: 6393 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 550 Data size: 5843 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: 60 Data size: 6393 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 60 Data size: 6393 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -697,7 +694,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@smb_bucket4_1 PREHOOK: Input: default@smb_bucket4_2 PREHOOK: Output: default@smb_join_results -[Error 30017]: Skipping 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 smb_join_results select /*+mapjoin(a)*/ * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key POSTHOOK: type: QUERY @@ -1221,7 +1217,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@smb_bucket4_1 PREHOOK: Input: default@smb_bucket4_2 PREHOOK: Output: default@normal_join_results -[Error 30017]: Skipping 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 normal_join_results select * from smb_bucket4_1 a full outer join smb_bucket4_2 b on a.key = b.key POSTHOOK: type: QUERY POSTHOOK: Input: default@smb_bucket4_1 diff --git ql/src/test/results/clientpositive/spark/smb_mapjoin_8.q.out ql/src/test/results/clientpositive/spark/smb_mapjoin_8.q.out index aada65b..802f8396 100644 --- ql/src/test/results/clientpositive/spark/smb_mapjoin_8.q.out +++ ql/src/test/results/clientpositive/spark/smb_mapjoin_8.q.out @@ -42,7 +42,6 @@ PREHOOK: query: insert overwrite table smb_bucket4_1 select * from smb_bucket_in PREHOOK: type: QUERY PREHOOK: Input: default@smb_bucket_input PREHOOK: Output: default@smb_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 smb_bucket4_1 select * from smb_bucket_input where key=4 or key=2000 or key=4000 POSTHOOK: type: QUERY POSTHOOK: Input: default@smb_bucket_input @@ -53,7 +52,6 @@ PREHOOK: query: insert overwrite table smb_bucket4_2 select * from smb_bucket_in PREHOOK: type: QUERY PREHOOK: Input: default@smb_bucket_input PREHOOK: Output: default@smb_bucket4_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 smb_bucket4_2 select * from smb_bucket_input where key=484 or key=3000 or key=5000 POSTHOOK: type: QUERY POSTHOOK: Input: default@smb_bucket_input @@ -96,7 +94,6 @@ PREHOOK: query: insert overwrite table smb_bucket4_1 select * from smb_bucket_in PREHOOK: type: QUERY PREHOOK: Input: default@smb_bucket_input PREHOOK: Output: default@smb_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 smb_bucket4_1 select * from smb_bucket_input where key=2000 or key=4000 POSTHOOK: type: QUERY POSTHOOK: Input: default@smb_bucket_input @@ -107,7 +104,6 @@ PREHOOK: query: insert overwrite table smb_bucket4_2 select * from smb_bucket_in PREHOOK: type: QUERY PREHOOK: Input: default@smb_bucket_input PREHOOK: Output: default@smb_bucket4_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 smb_bucket4_2 select * from smb_bucket_input where key=3000 or key=5000 POSTHOOK: type: QUERY POSTHOOK: Input: default@smb_bucket_input @@ -146,7 +142,6 @@ PREHOOK: query: insert overwrite table smb_bucket4_1 select * from smb_bucket_in PREHOOK: type: QUERY PREHOOK: Input: default@smb_bucket_input PREHOOK: Output: default@smb_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 smb_bucket4_1 select * from smb_bucket_input where key=4000 POSTHOOK: type: QUERY POSTHOOK: Input: default@smb_bucket_input @@ -157,7 +152,6 @@ PREHOOK: query: insert overwrite table smb_bucket4_2 select * from smb_bucket_in PREHOOK: type: QUERY PREHOOK: Input: default@smb_bucket_input PREHOOK: Output: default@smb_bucket4_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 smb_bucket4_2 select * from smb_bucket_input where key=5000 POSTHOOK: type: QUERY POSTHOOK: Input: default@smb_bucket_input @@ -192,7 +186,6 @@ PREHOOK: query: insert overwrite table smb_bucket4_1 select * from smb_bucket_in PREHOOK: type: QUERY PREHOOK: Input: default@smb_bucket_input PREHOOK: Output: default@smb_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 smb_bucket4_1 select * from smb_bucket_input where key=1000 or key=4000 POSTHOOK: type: QUERY POSTHOOK: Input: default@smb_bucket_input @@ -203,7 +196,6 @@ PREHOOK: query: insert overwrite table smb_bucket4_2 select * from smb_bucket_in PREHOOK: type: QUERY PREHOOK: Input: default@smb_bucket_input PREHOOK: Output: default@smb_bucket4_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 smb_bucket4_2 select * from smb_bucket_input where key=1000 or key=5000 POSTHOOK: type: QUERY POSTHOOK: Input: default@smb_bucket_input @@ -240,7 +232,6 @@ PREHOOK: query: insert overwrite table smb_bucket4_1 select * from smb_bucket_in PREHOOK: type: QUERY PREHOOK: Input: default@smb_bucket_input PREHOOK: Output: default@smb_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 smb_bucket4_1 select * from smb_bucket_input where key=1000 or key=4000 POSTHOOK: type: QUERY POSTHOOK: Input: default@smb_bucket_input @@ -251,7 +242,6 @@ PREHOOK: query: insert overwrite table smb_bucket4_2 select * from smb_bucket_in PREHOOK: type: QUERY PREHOOK: Input: default@smb_bucket_input PREHOOK: Output: default@smb_bucket4_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 smb_bucket4_2 select * from smb_bucket_input where key=1000 or key=5000 POSTHOOK: type: QUERY POSTHOOK: Input: default@smb_bucket_input @@ -262,7 +252,6 @@ PREHOOK: query: insert overwrite table smb_bucket4_3 select * from smb_bucket_in PREHOOK: type: QUERY PREHOOK: Input: default@smb_bucket_input PREHOOK: Output: default@smb_bucket4_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 smb_bucket4_3 select * from smb_bucket_input where key=1000 or key=5000 POSTHOOK: type: QUERY POSTHOOK: Input: default@smb_bucket_input @@ -291,7 +280,6 @@ PREHOOK: query: insert overwrite table smb_bucket4_1 select * from smb_bucket_in PREHOOK: type: QUERY PREHOOK: Input: default@smb_bucket_input PREHOOK: Output: default@smb_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 smb_bucket4_1 select * from smb_bucket_input where key=1000 or key=4000 POSTHOOK: type: QUERY POSTHOOK: Input: default@smb_bucket_input @@ -302,7 +290,6 @@ PREHOOK: query: insert overwrite table smb_bucket4_2 select * from smb_bucket_in PREHOOK: type: QUERY PREHOOK: Input: default@smb_bucket_input PREHOOK: Output: default@smb_bucket4_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 smb_bucket4_2 select * from smb_bucket_input where key=1000 or key=5000 POSTHOOK: type: QUERY POSTHOOK: Input: default@smb_bucket_input @@ -313,7 +300,6 @@ PREHOOK: query: insert overwrite table smb_bucket4_3 select * from smb_bucket_in PREHOOK: type: QUERY PREHOOK: Input: default@smb_bucket_input PREHOOK: Output: default@smb_bucket4_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 smb_bucket4_3 select * from smb_bucket_input where key=1000 or key=4000 POSTHOOK: type: QUERY POSTHOOK: Input: default@smb_bucket_input @@ -341,7 +327,6 @@ PREHOOK: query: insert overwrite table smb_bucket4_1 select * from smb_bucket_in PREHOOK: type: QUERY PREHOOK: Input: default@smb_bucket_input PREHOOK: Output: default@smb_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 smb_bucket4_1 select * from smb_bucket_input where key=4000 POSTHOOK: type: QUERY POSTHOOK: Input: default@smb_bucket_input @@ -352,7 +337,6 @@ PREHOOK: query: insert overwrite table smb_bucket4_2 select * from smb_bucket_in PREHOOK: type: QUERY PREHOOK: Input: default@smb_bucket_input PREHOOK: Output: default@smb_bucket4_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 smb_bucket4_2 select * from smb_bucket_input where key=5000 POSTHOOK: type: QUERY POSTHOOK: Input: default@smb_bucket_input @@ -363,7 +347,6 @@ PREHOOK: query: insert overwrite table smb_bucket4_3 select * from smb_bucket_in PREHOOK: type: QUERY PREHOOK: Input: default@smb_bucket_input PREHOOK: Output: default@smb_bucket4_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 smb_bucket4_3 select * from smb_bucket_input where key=4000 POSTHOOK: type: QUERY POSTHOOK: Input: default@smb_bucket_input @@ -390,7 +373,6 @@ PREHOOK: query: insert overwrite table smb_bucket4_1 select * from smb_bucket_in PREHOOK: type: QUERY PREHOOK: Input: default@smb_bucket_input PREHOOK: Output: default@smb_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 smb_bucket4_1 select * from smb_bucket_input where key=00000 POSTHOOK: type: QUERY POSTHOOK: Input: default@smb_bucket_input @@ -401,7 +383,6 @@ PREHOOK: query: insert overwrite table smb_bucket4_2 select * from smb_bucket_in PREHOOK: type: QUERY PREHOOK: Input: default@smb_bucket_input PREHOOK: Output: default@smb_bucket4_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 smb_bucket4_2 select * from smb_bucket_input where key=4000 POSTHOOK: type: QUERY POSTHOOK: Input: default@smb_bucket_input @@ -412,7 +393,6 @@ PREHOOK: query: insert overwrite table smb_bucket4_3 select * from smb_bucket_in PREHOOK: type: QUERY PREHOOK: Input: default@smb_bucket_input PREHOOK: Output: default@smb_bucket4_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 smb_bucket4_3 select * from smb_bucket_input where key=5000 POSTHOOK: type: QUERY POSTHOOK: Input: default@smb_bucket_input @@ -439,7 +419,6 @@ PREHOOK: query: insert overwrite table smb_bucket4_1 select * from smb_bucket_in PREHOOK: type: QUERY PREHOOK: Input: default@smb_bucket_input PREHOOK: Output: default@smb_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 smb_bucket4_1 select * from smb_bucket_input where key=1000 POSTHOOK: type: QUERY POSTHOOK: Input: default@smb_bucket_input @@ -450,7 +429,6 @@ PREHOOK: query: insert overwrite table smb_bucket4_2 select * from smb_bucket_in PREHOOK: type: QUERY PREHOOK: Input: default@smb_bucket_input PREHOOK: Output: default@smb_bucket4_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 smb_bucket4_2 select * from smb_bucket_input where key=4000 POSTHOOK: type: QUERY POSTHOOK: Input: default@smb_bucket_input @@ -461,7 +439,6 @@ PREHOOK: query: insert overwrite table smb_bucket4_3 select * from smb_bucket_in PREHOOK: type: QUERY PREHOOK: Input: default@smb_bucket_input PREHOOK: Output: default@smb_bucket4_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 smb_bucket4_3 select * from smb_bucket_input where key=5000 POSTHOOK: type: QUERY POSTHOOK: Input: default@smb_bucket_input diff --git ql/src/test/results/clientpositive/spark/sort_merge_join_desc_1.q.out ql/src/test/results/clientpositive/spark/sort_merge_join_desc_1.q.out index 32c3818..0142b02 100644 --- ql/src/test/results/clientpositive/spark/sort_merge_join_desc_1.q.out +++ ql/src/test/results/clientpositive/spark/sort_merge_join_desc_1.q.out @@ -26,7 +26,6 @@ PREHOOK: query: insert overwrite table table_desc1 select key, value from src PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@table_desc1 -[Error 30017]: Skipping 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_desc1 select key, value from src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -37,7 +36,6 @@ PREHOOK: query: insert overwrite table table_desc2 select key, value from src PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@table_desc2 -[Error 30017]: Skipping 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_desc2 select key, value from src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -72,28 +70,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 58 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 and (key < 10)) (type: boolean) - Statistics: Num rows: 9 Data size: 901 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: string) sort order: + Map-reduce partition columns: key (type: string) - Statistics: Num rows: 9 Data size: 901 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE Map 4 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 58 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 and (key < 10)) (type: boolean) - Statistics: Num rows: 9 Data size: 901 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: string) sort order: + Map-reduce partition columns: key (type: string) - Statistics: Num rows: 9 Data size: 901 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 83 Data size: 881 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: 9 Data size: 991 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 91 Data size: 969 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 9 Data size: 991 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 91 Data size: 969 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash diff --git ql/src/test/results/clientpositive/spark/sort_merge_join_desc_2.q.out ql/src/test/results/clientpositive/spark/sort_merge_join_desc_2.q.out index ae08516..d88c6f0 100644 --- ql/src/test/results/clientpositive/spark/sort_merge_join_desc_2.q.out +++ ql/src/test/results/clientpositive/spark/sort_merge_join_desc_2.q.out @@ -30,7 +30,6 @@ PREHOOK: query: insert overwrite table table_desc1 select key, value from src PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@table_desc1 -[Error 30017]: Skipping 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_desc1 select key, value from src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -41,7 +40,6 @@ PREHOOK: query: insert overwrite table table_desc2 select key, value from src PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@table_desc2 -[Error 30017]: Skipping 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_desc2 select key, value from src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -80,28 +78,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 29 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 and value is not null) and (key < 10)) (type: boolean) - Statistics: Num rows: 2 Data size: 400 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 41 Data size: 435 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: string), value (type: string) sort order: ++ Map-reduce partition columns: key (type: string), value (type: string) - Statistics: Num rows: 2 Data size: 400 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 41 Data size: 435 Basic stats: COMPLETE Column stats: NONE Map 4 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 29 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 and value is not null) and (key < 10)) (type: boolean) - Statistics: Num rows: 2 Data size: 400 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 41 Data size: 435 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: string), value (type: string) sort order: ++ Map-reduce partition columns: key (type: string), value (type: string) - Statistics: Num rows: 2 Data size: 400 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 41 Data size: 435 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator @@ -110,9 +108,9 @@ STAGE PLANS: condition expressions: 0 1 - Statistics: Num rows: 2 Data size: 440 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 45 Data size: 478 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 2 Data size: 440 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 45 Data size: 478 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash diff --git ql/src/test/results/clientpositive/spark/sort_merge_join_desc_3.q.out ql/src/test/results/clientpositive/spark/sort_merge_join_desc_3.q.out index 6add9f9..51da7c5 100644 --- ql/src/test/results/clientpositive/spark/sort_merge_join_desc_3.q.out +++ ql/src/test/results/clientpositive/spark/sort_merge_join_desc_3.q.out @@ -30,7 +30,6 @@ PREHOOK: query: insert overwrite table table_desc1 select key, value from src PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@table_desc1 -[Error 30017]: Skipping 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_desc1 select key, value from src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -41,7 +40,6 @@ PREHOOK: query: insert overwrite table table_desc2 select key, value from src PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@table_desc2 -[Error 30017]: Skipping 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_desc2 select key, value from src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -80,28 +78,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 29 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 and value is not null) and (key < 10)) (type: boolean) - Statistics: Num rows: 2 Data size: 400 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 41 Data size: 435 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: string), value (type: string) sort order: ++ Map-reduce partition columns: key (type: string), value (type: string) - Statistics: Num rows: 2 Data size: 400 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 41 Data size: 435 Basic stats: COMPLETE Column stats: NONE Map 4 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 29 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 and value is not null) and (key < 10)) (type: boolean) - Statistics: Num rows: 2 Data size: 400 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 41 Data size: 435 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: string), value (type: string) sort order: ++ Map-reduce partition columns: key (type: string), value (type: string) - Statistics: Num rows: 2 Data size: 400 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 41 Data size: 435 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator @@ -110,9 +108,9 @@ STAGE PLANS: condition expressions: 0 1 - Statistics: Num rows: 2 Data size: 440 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 45 Data size: 478 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 2 Data size: 440 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 45 Data size: 478 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash diff --git ql/src/test/results/clientpositive/spark/sort_merge_join_desc_4.q.out ql/src/test/results/clientpositive/spark/sort_merge_join_desc_4.q.out index b810a56..f52cc6d 100644 --- ql/src/test/results/clientpositive/spark/sort_merge_join_desc_4.q.out +++ ql/src/test/results/clientpositive/spark/sort_merge_join_desc_4.q.out @@ -30,7 +30,6 @@ PREHOOK: query: insert overwrite table table_desc1 select key, value from src PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@table_desc1 -[Error 30017]: Skipping 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_desc1 select key, value from src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -41,7 +40,6 @@ PREHOOK: query: insert overwrite table table_desc2 select key, value from src PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@table_desc2 -[Error 30017]: Skipping 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_desc2 select key, value from src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -78,28 +76,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 29 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 and value is not null) and (key < 10)) (type: boolean) - Statistics: Num rows: 2 Data size: 400 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 41 Data size: 435 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: string), value (type: string) sort order: ++ Map-reduce partition columns: key (type: string), value (type: string) - Statistics: Num rows: 2 Data size: 400 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 41 Data size: 435 Basic stats: COMPLETE Column stats: NONE Map 4 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 29 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 and value is not null) and (key < 10)) (type: boolean) - Statistics: Num rows: 2 Data size: 400 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 41 Data size: 435 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: string), value (type: string) sort order: ++ Map-reduce partition columns: key (type: string), value (type: string) - Statistics: Num rows: 2 Data size: 400 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 41 Data size: 435 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator @@ -108,9 +106,9 @@ STAGE PLANS: condition expressions: 0 1 - Statistics: Num rows: 2 Data size: 440 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 45 Data size: 478 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 2 Data size: 440 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 45 Data size: 478 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash diff --git ql/src/test/results/clientpositive/spark/sort_merge_join_desc_5.q.out ql/src/test/results/clientpositive/spark/sort_merge_join_desc_5.q.out index f59d942..864858a 100644 --- ql/src/test/results/clientpositive/spark/sort_merge_join_desc_5.q.out +++ ql/src/test/results/clientpositive/spark/sort_merge_join_desc_5.q.out @@ -12,7 +12,6 @@ PREHOOK: query: INSERT OVERWRITE TABLE srcbucket_mapjoin_part_1 PARTITION (part= PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@srcbucket_mapjoin_part_1@part=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 srcbucket_mapjoin_part_1 PARTITION (part='1') SELECT * FROM src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -33,7 +32,6 @@ PREHOOK: query: INSERT OVERWRITE TABLE srcbucket_mapjoin_part_2 PARTITION (part= PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@srcbucket_mapjoin_part_2@part=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 srcbucket_mapjoin_part_2 PARTITION (part='1') SELECT * FROM src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -129,17 +127,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - 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 GatherStats: false Filter Operator isSamplingPred: false 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 tag: 1 auto parallelism: true Path -> Alias: @@ -162,10 +160,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.srcbucket_mapjoin_part_2 numFiles 1 - numRows -1 + numRows 500 partition_columns part partition_columns.types string - rawDataSize -1 + rawDataSize 5312 serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -199,17 +197,17 @@ 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 GatherStats: false Filter Operator isSamplingPred: false 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 tag: 0 auto parallelism: true Path -> Alias: @@ -232,10 +230,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.srcbucket_mapjoin_part_1 numFiles 1 - numRows -1 + numRows 500 partition_columns part partition_columns.types string - rawDataSize -1 + rawDataSize 5312 serialization.ddl struct srcbucket_mapjoin_part_1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -274,9 +272,9 @@ STAGE PLANS: condition expressions: 0 1 - Statistics: Num rows: 799 Data size: 3198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 799 Data size: 3198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash diff --git ql/src/test/results/clientpositive/spark/sort_merge_join_desc_6.q.out ql/src/test/results/clientpositive/spark/sort_merge_join_desc_6.q.out index 4085d9a..8e77e12 100644 --- ql/src/test/results/clientpositive/spark/sort_merge_join_desc_6.q.out +++ ql/src/test/results/clientpositive/spark/sort_merge_join_desc_6.q.out @@ -12,7 +12,6 @@ PREHOOK: query: INSERT OVERWRITE TABLE srcbucket_mapjoin_part_1 PARTITION (part= PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@srcbucket_mapjoin_part_1@part=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 srcbucket_mapjoin_part_1 PARTITION (part='1') SELECT * FROM src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -33,7 +32,6 @@ PREHOOK: query: INSERT OVERWRITE TABLE srcbucket_mapjoin_part_2 PARTITION (part= PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@srcbucket_mapjoin_part_2@part=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 srcbucket_mapjoin_part_2 PARTITION (part='1') SELECT * FROM src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -129,17 +127,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - 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 GatherStats: false Filter Operator isSamplingPred: false 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 tag: 1 auto parallelism: true Path -> Alias: @@ -162,10 +160,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.srcbucket_mapjoin_part_2 numFiles 2 - numRows -1 + numRows 500 partition_columns part partition_columns.types string - rawDataSize -1 + rawDataSize 5312 serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -198,17 +196,17 @@ 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 GatherStats: false Filter Operator isSamplingPred: false 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 tag: 0 auto parallelism: true Path -> Alias: @@ -231,10 +229,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.srcbucket_mapjoin_part_1 numFiles 2 - numRows -1 + numRows 500 partition_columns part partition_columns.types string - rawDataSize -1 + rawDataSize 5312 serialization.ddl struct srcbucket_mapjoin_part_1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -273,9 +271,9 @@ STAGE PLANS: condition expressions: 0 1 - Statistics: Num rows: 799 Data size: 3198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 799 Data size: 3198 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash diff --git ql/src/test/results/clientpositive/spark/sort_merge_join_desc_7.q.out ql/src/test/results/clientpositive/spark/sort_merge_join_desc_7.q.out index 28336c5..0b16452 100644 --- ql/src/test/results/clientpositive/spark/sort_merge_join_desc_7.q.out +++ ql/src/test/results/clientpositive/spark/sort_merge_join_desc_7.q.out @@ -12,7 +12,6 @@ PREHOOK: query: INSERT OVERWRITE TABLE srcbucket_mapjoin_part_1 PARTITION (part= PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@srcbucket_mapjoin_part_1@part=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 srcbucket_mapjoin_part_1 PARTITION (part='1') SELECT * FROM src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -31,7 +30,6 @@ PREHOOK: query: INSERT OVERWRITE TABLE srcbucket_mapjoin_part_1 PARTITION (part= PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@srcbucket_mapjoin_part_1@part=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 srcbucket_mapjoin_part_1 PARTITION (part='2') SELECT * FROM src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -52,7 +50,6 @@ PREHOOK: query: INSERT OVERWRITE TABLE srcbucket_mapjoin_part_2 PARTITION (part= PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@srcbucket_mapjoin_part_2@part=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 srcbucket_mapjoin_part_2 PARTITION (part='1') SELECT * FROM src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -71,7 +68,6 @@ PREHOOK: query: INSERT OVERWRITE TABLE srcbucket_mapjoin_part_2 PARTITION (part= PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@srcbucket_mapjoin_part_2@part=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 srcbucket_mapjoin_part_2 PARTITION (part='2') SELECT * FROM src POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -167,17 +163,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 2906 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: key is not null (type: boolean) - 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 tag: 1 auto parallelism: true Path -> Alias: @@ -200,10 +196,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.srcbucket_mapjoin_part_2 numFiles 2 - numRows -1 + numRows 500 partition_columns part partition_columns.types string - rawDataSize -1 + rawDataSize 5312 serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -247,10 +243,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.srcbucket_mapjoin_part_2 numFiles 2 - numRows -1 + numRows 500 partition_columns part partition_columns.types string - rawDataSize -1 + rawDataSize 5312 serialization.ddl struct srcbucket_mapjoin_part_2 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -284,17 +280,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: a - Statistics: Num rows: 2906 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: key is not null (type: boolean) - 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 Reduce Output Operator key expressions: key (type: int) sort order: + Map-reduce partition columns: key (type: int) - 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 tag: 0 auto parallelism: true Path -> Alias: @@ -317,10 +313,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.srcbucket_mapjoin_part_1 numFiles 2 - numRows -1 + numRows 500 partition_columns part partition_columns.types string - rawDataSize -1 + rawDataSize 5312 serialization.ddl struct srcbucket_mapjoin_part_1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -364,10 +360,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.srcbucket_mapjoin_part_1 numFiles 2 - numRows -1 + numRows 500 partition_columns part partition_columns.types string - rawDataSize -1 + rawDataSize 5312 serialization.ddl struct srcbucket_mapjoin_part_1 { i32 key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -406,9 +402,9 @@ STAGE PLANS: condition expressions: 0 1 - Statistics: Num rows: 1598 Data size: 6393 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 1598 Data size: 6393 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash diff --git ql/src/test/results/clientpositive/spark/sort_merge_join_desc_8.q.out ql/src/test/results/clientpositive/spark/sort_merge_join_desc_8.q.out index 087a89d..3208610 100644 --- ql/src/test/results/clientpositive/spark/sort_merge_join_desc_8.q.out +++ ql/src/test/results/clientpositive/spark/sort_merge_join_desc_8.q.out @@ -58,7 +58,6 @@ PREHOOK: query: insert overwrite table table_desc1 select key, value from src so PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@table_desc1 -[Error 30017]: Skipping 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_desc1 select key, value from src sort by key DESC POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -69,7 +68,6 @@ PREHOOK: query: insert overwrite table table_desc2 select key, value from src so PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@table_desc2 -[Error 30017]: Skipping 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_desc2 select key, value from src sort by key DESC POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -80,7 +78,6 @@ PREHOOK: query: insert overwrite table table_desc3 select key, value, concat(val PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@table_desc3 -[Error 30017]: Skipping 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_desc3 select key, value, concat(value,"_2") as value2 from src sort by key, value, value2 DESC POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -92,7 +89,6 @@ PREHOOK: query: insert overwrite table table_desc4 select key, concat(value,"_2" PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@table_desc4 -[Error 30017]: Skipping 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_desc4 select key, concat(value,"_2") as value2 from src sort by key, value2 DESC POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -127,28 +123,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 58 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 and (key < 10)) (type: boolean) - Statistics: Num rows: 9 Data size: 901 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: string) sort order: + Map-reduce partition columns: key (type: string) - Statistics: Num rows: 9 Data size: 901 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE Map 4 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 58 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 and (key < 10)) (type: boolean) - Statistics: Num rows: 9 Data size: 901 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: string) sort order: + Map-reduce partition columns: key (type: string) - Statistics: Num rows: 9 Data size: 901 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator @@ -157,9 +153,9 @@ STAGE PLANS: condition expressions: 0 1 - Statistics: Num rows: 9 Data size: 991 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 91 Data size: 969 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 9 Data size: 991 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 91 Data size: 969 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash @@ -235,28 +231,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: b - Statistics: Num rows: 34 Data size: 6812 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 500 Data size: 6312 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: ((key is not null and value2 is not null) and (key < 10)) (type: boolean) - Statistics: Num rows: 3 Data size: 601 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 41 Data size: 517 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: string), value2 (type: string) sort order: ++ Map-reduce partition columns: key (type: string), value2 (type: string) - Statistics: Num rows: 3 Data size: 601 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 41 Data size: 517 Basic stats: COMPLETE Column stats: NONE Map 4 Map Operator Tree: TableScan alias: a - Statistics: Num rows: 53 Data size: 10718 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 500 Data size: 10218 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: ((key is not null and value2 is not null) and (key < 10)) (type: boolean) - Statistics: Num rows: 4 Data size: 808 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 41 Data size: 837 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: string), value2 (type: string) sort order: ++ Map-reduce partition columns: key (type: string), value2 (type: string) - Statistics: Num rows: 4 Data size: 808 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 41 Data size: 837 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator @@ -265,9 +261,9 @@ STAGE PLANS: condition expressions: 0 1 - Statistics: Num rows: 4 Data size: 888 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 45 Data size: 920 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 4 Data size: 888 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 45 Data size: 920 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash 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/temp_table.q.out ql/src/test/results/clientpositive/spark/temp_table.q.out index a126fc7..7215057 100644 --- ql/src/test/results/clientpositive/spark/temp_table.q.out +++ ql/src/test/results/clientpositive/spark/temp_table.q.out @@ -63,7 +63,6 @@ PREHOOK: type: CREATETABLE_AS_SELECT PREHOOK: Input: default@src PREHOOK: Output: database:default PREHOOK: Output: default@foo -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: CREATE TEMPORARY TABLE foo AS SELECT * FROM src WHERE key % 2 = 0 POSTHOOK: type: CREATETABLE_AS_SELECT POSTHOOK: Input: default@src @@ -134,7 +133,6 @@ PREHOOK: type: CREATETABLE_AS_SELECT PREHOOK: Input: default@src PREHOOK: Output: database:default PREHOOK: Output: default@bar -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: CREATE TEMPORARY TABLE bar AS SELECT * FROM src WHERE key % 2 = 1 POSTHOOK: type: CREATETABLE_AS_SELECT POSTHOOK: Input: default@src @@ -170,14 +168,14 @@ STAGE PLANS: Processor Tree: TableScan alias: foo - Statistics: Num rows: 14 Data size: 2856 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 247 Data size: 2609 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string), value (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 14 Data size: 2856 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 247 Data size: 2609 Basic stats: COMPLETE Column stats: NONE Limit Number of rows: 10 - Statistics: Num rows: 10 Data size: 2040 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 100 Basic stats: COMPLETE Column stats: NONE ListSink PREHOOK: query: select * from foo limit 10 @@ -247,13 +245,13 @@ STAGE PLANS: Select Operator expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 28 Data size: 5812 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Limit Number of rows: 10 - Statistics: Num rows: 10 Data size: 2070 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 100 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 10 Data size: 2070 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 100 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -299,7 +297,6 @@ PREHOOK: query: INSERT OVERWRITE TABLE baz SELECT * from foo PREHOOK: type: QUERY PREHOOK: Input: default@foo PREHOOK: Output: default@baz -[Error 30017]: Skipping 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 baz SELECT * from foo POSTHOOK: type: QUERY POSTHOOK: Input: default@foo @@ -326,7 +323,6 @@ PREHOOK: query: INSERT OVERWRITE TABLE bay SELECT * FROM src ORDER BY key PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@bay -[Error 30017]: Skipping 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 bay SELECT * FROM src ORDER BY key POSTHOOK: type: QUERY POSTHOOK: Input: default@src @@ -389,7 +385,6 @@ PREHOOK: type: CREATETABLE_AS_SELECT PREHOOK: Input: default@foo PREHOOK: Output: database:two PREHOOK: Output: two@foo -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: CREATE TEMPORARY TABLE foo AS SELECT * FROM default.foo POSTHOOK: type: CREATETABLE_AS_SELECT POSTHOOK: Input: default@foo diff --git ql/src/test/results/clientpositive/spark/temp_table_join1.q.out ql/src/test/results/clientpositive/spark/temp_table_join1.q.out index f06a9ab..7e78ed5 100644 --- ql/src/test/results/clientpositive/spark/temp_table_join1.q.out +++ ql/src/test/results/clientpositive/spark/temp_table_join1.q.out @@ -3,7 +3,6 @@ PREHOOK: type: CREATETABLE_AS_SELECT PREHOOK: Input: default@src PREHOOK: Output: database:default PREHOOK: Output: default@src_nontemp -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: CREATE TABLE src_nontemp AS SELECT * FROM src limit 10 POSTHOOK: type: CREATETABLE_AS_SELECT POSTHOOK: Input: default@src @@ -14,7 +13,6 @@ PREHOOK: type: CREATETABLE_AS_SELECT PREHOOK: Input: default@src PREHOOK: Output: database:default PREHOOK: Output: default@src_temp -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: CREATE TEMPORARY TABLE src_temp AS SELECT * FROM src limit 10 POSTHOOK: type: CREATETABLE_AS_SELECT POSTHOOK: Input: default@src @@ -45,29 +43,29 @@ STAGE PLANS: Map Operator Tree: TableScan alias: src2 - Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 10 Data size: 104 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: 52 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: string) sort order: + Map-reduce partition columns: key (type: string) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 5 Data size: 52 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Map 3 Map Operator Tree: TableScan alias: src1 - Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 52 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: string) sort order: + Map-reduce partition columns: key (type: string) - Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 52 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator @@ -77,14 +75,14 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} 1 {VALUE._col0} outputColumnNames: _col0, _col6 - Statistics: Num rows: 1 Data size: 125 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 57 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col6 (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 125 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 57 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 125 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 57 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -141,29 +139,29 @@ STAGE PLANS: Map Operator Tree: TableScan alias: src2 - Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 10 Data size: 104 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: 52 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: string) sort order: + Map-reduce partition columns: key (type: string) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 5 Data size: 52 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Map 3 Map Operator Tree: TableScan alias: src1 - Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 52 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: string) sort order: + Map-reduce partition columns: key (type: string) - Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 52 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator @@ -173,14 +171,14 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} 1 {VALUE._col0} outputColumnNames: _col0, _col6 - Statistics: Num rows: 1 Data size: 125 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 57 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col6 (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 125 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 57 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 125 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 57 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -239,29 +237,29 @@ STAGE PLANS: Map Operator Tree: TableScan alias: src2 - Statistics: Num rows: 0 Data size: 114 Basic stats: PARTIAL Column stats: NONE + Statistics: Num rows: 10 Data size: 104 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: 52 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: string) sort order: + Map-reduce partition columns: key (type: string) - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Statistics: Num rows: 5 Data size: 52 Basic stats: COMPLETE Column stats: NONE value expressions: value (type: string) Map 3 Map Operator Tree: TableScan alias: src1 - Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 104 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 52 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: key (type: string) sort order: + Map-reduce partition columns: key (type: string) - Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 52 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: Join Operator @@ -271,14 +269,14 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} 1 {VALUE._col0} outputColumnNames: _col0, _col6 - Statistics: Num rows: 1 Data size: 125 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 57 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col6 (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 125 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 57 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 125 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 5 Data size: 57 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/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_between_in.q.out ql/src/test/results/clientpositive/spark/vector_between_in.q.out index f0d2ac7..7d1323b 100644 --- ql/src/test/results/clientpositive/spark/vector_between_in.q.out +++ ql/src/test/results/clientpositive/spark/vector_between_in.q.out @@ -3,7 +3,6 @@ PREHOOK: type: CREATETABLE_AS_SELECT PREHOOK: Input: default@alltypesorc PREHOOK: Output: database:default PREHOOK: Output: default@decimal_date_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: CREATE TABLE decimal_date_test STORED AS ORC AS SELECT cdouble, CAST (((cdouble*22.1)/37) AS DECIMAL(20,10)) AS cdecimal1, CAST (((cdouble*9.3)/13) AS DECIMAL(23,14)) AS cdecimal2, CAST(CAST((CAST(cint AS BIGINT) *ctinyint) AS TIMESTAMP) AS DATE) AS cdate FROM alltypesorc ORDER BY cdate POSTHOOK: type: CREATETABLE_AS_SELECT POSTHOOK: Input: default@alltypesorc @@ -28,28 +27,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: decimal_date_test - Statistics: Num rows: 1952 Data size: 109320 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 12288 Data size: 2467616 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (cdate) IN (1969-10-26, 1969-07-14) (type: boolean) - Statistics: Num rows: 976 Data size: 54660 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: cdate (type: date) outputColumnNames: _col0 - Statistics: Num rows: 976 Data size: 54660 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: date) sort order: + - Statistics: Num rows: 976 Data size: 54660 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Reducer 2 Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey0 (type: date) outputColumnNames: _col0 - Statistics: Num rows: 976 Data size: 54660 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 976 Data size: 54660 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -81,12 +80,12 @@ STAGE PLANS: Map Operator Tree: TableScan alias: decimal_date_test - Statistics: Num rows: 1952 Data size: 109320 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 12288 Data size: 2467616 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (not (cdate) IN (1969-10-26, 1969-07-14, 1970-01-21)) (type: boolean) - Statistics: Num rows: 976 Data size: 54660 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 976 Data size: 54660 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash @@ -142,28 +141,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: decimal_date_test - Statistics: Num rows: 976 Data size: 109320 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 12288 Data size: 2467616 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (cdecimal1) IN (2365.8945945946, 881.0135135135, -3367.6517567568) (type: boolean) - Statistics: Num rows: 488 Data size: 54660 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: cdecimal1 (type: decimal(20,10)) outputColumnNames: _col0 - Statistics: Num rows: 488 Data size: 54660 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: decimal(20,10)) sort order: + - Statistics: Num rows: 488 Data size: 54660 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Reducer 2 Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey0 (type: decimal(20,10)) outputColumnNames: _col0 - Statistics: Num rows: 488 Data size: 54660 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 488 Data size: 54660 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -195,12 +194,12 @@ STAGE PLANS: Map Operator Tree: TableScan alias: decimal_date_test - Statistics: Num rows: 976 Data size: 109320 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 12288 Data size: 2467616 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (not (cdecimal1) IN (2365.8945945946, 881.0135135135, -3367.6517567568)) (type: boolean) - Statistics: Num rows: 488 Data size: 54660 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 488 Data size: 54660 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash @@ -256,28 +255,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: decimal_date_test - Statistics: Num rows: 1952 Data size: 109320 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 12288 Data size: 2467616 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: cdate BETWEEN 1969-12-30 AND 1970-01-02 (type: boolean) - Statistics: Num rows: 976 Data size: 54660 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: cdate (type: date) outputColumnNames: _col0 - Statistics: Num rows: 976 Data size: 54660 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: date) sort order: + - Statistics: Num rows: 976 Data size: 54660 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Reducer 2 Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey0 (type: date) outputColumnNames: _col0 - Statistics: Num rows: 976 Data size: 54660 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 976 Data size: 54660 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -309,28 +308,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: decimal_date_test - Statistics: Num rows: 1952 Data size: 109320 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 12288 Data size: 2467616 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: cdate NOT BETWEEN 1968-05-01 AND 1971-09-01 (type: boolean) - Statistics: Num rows: 976 Data size: 54660 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: cdate (type: date) outputColumnNames: _col0 - Statistics: Num rows: 976 Data size: 54660 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: date) sort order: + - Statistics: Num rows: 976 Data size: 54660 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Reducer 2 Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey0 (type: date) outputColumnNames: _col0 - Statistics: Num rows: 976 Data size: 54660 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 976 Data size: 54660 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -362,28 +361,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: decimal_date_test - Statistics: Num rows: 976 Data size: 109320 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 12288 Data size: 2467616 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: cdecimal1 BETWEEN -20 AND 45.9918918919 (type: boolean) - Statistics: Num rows: 488 Data size: 54660 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: cdecimal1 (type: decimal(20,10)) outputColumnNames: _col0 - Statistics: Num rows: 488 Data size: 54660 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: decimal(20,10)) sort order: + - Statistics: Num rows: 488 Data size: 54660 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Reducer 2 Reduce Operator Tree: Select Operator expressions: KEY.reducesinkkey0 (type: decimal(20,10)) outputColumnNames: _col0 - Statistics: Num rows: 488 Data size: 54660 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 488 Data size: 54660 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -415,12 +414,12 @@ STAGE PLANS: Map Operator Tree: TableScan alias: decimal_date_test - Statistics: Num rows: 976 Data size: 109320 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 12288 Data size: 2467616 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: cdecimal1 NOT BETWEEN -2000 AND 4390.1351351351 (type: boolean) - Statistics: Num rows: 488 Data size: 54660 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 488 Data size: 54660 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6144 Data size: 1233808 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() mode: hash 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_decimal_aggregate.q.out ql/src/test/results/clientpositive/spark/vector_decimal_aggregate.q.out index 3c6d561..c7d16a4 100644 --- ql/src/test/results/clientpositive/spark/vector_decimal_aggregate.q.out +++ ql/src/test/results/clientpositive/spark/vector_decimal_aggregate.q.out @@ -7,7 +7,6 @@ PREHOOK: type: CREATETABLE_AS_SELECT PREHOOK: Input: default@alltypesorc PREHOOK: Output: database:default PREHOOK: Output: default@decimal_vgby -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: CREATE TABLE decimal_vgby STORED AS ORC AS SELECT cdouble, CAST (((cdouble*22.1)/37) AS DECIMAL(20,10)) AS cdecimal1, CAST (((cdouble*9.3)/13) AS DECIMAL(23,14)) AS cdecimal2, @@ -48,22 +47,22 @@ STAGE PLANS: Map Operator Tree: TableScan alias: decimal_vgby - Statistics: Num rows: 559 Data size: 127658 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 12288 Data size: 2165060 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: cint (type: int), cdecimal1 (type: decimal(20,10)), cdecimal2 (type: decimal(23,14)) outputColumnNames: cint, cdecimal1, cdecimal2 - Statistics: Num rows: 559 Data size: 127658 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 12288 Data size: 2165060 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(cdecimal1), max(cdecimal1), min(cdecimal1), sum(cdecimal1), count(cdecimal2), max(cdecimal2), min(cdecimal2), sum(cdecimal2), count() keys: cint (type: int) mode: hash outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9 - Statistics: Num rows: 559 Data size: 127658 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 12288 Data size: 2165060 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: 559 Data size: 127658 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 12288 Data size: 2165060 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint), _col2 (type: decimal(20,10)), _col3 (type: decimal(20,10)), _col4 (type: decimal(30,10)), _col5 (type: bigint), _col6 (type: decimal(23,14)), _col7 (type: decimal(23,14)), _col8 (type: decimal(33,14)), _col9 (type: bigint) Execution mode: vectorized Reducer 2 @@ -73,17 +72,17 @@ STAGE PLANS: keys: KEY._col0 (type: int) mode: mergepartial outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9 - Statistics: Num rows: 279 Data size: 63714 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6144 Data size: 1082530 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (_col9 > 1) (type: boolean) - Statistics: Num rows: 93 Data size: 21238 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2048 Data size: 360843 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: decimal(20,10)), _col3 (type: decimal(20,10)), _col4 (type: decimal(30,10)), _col5 (type: bigint), _col6 (type: decimal(23,14)), _col7 (type: decimal(23,14)), _col8 (type: decimal(33,14)) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 93 Data size: 21238 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2048 Data size: 360843 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 93 Data size: 21238 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2048 Data size: 360843 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -153,22 +152,22 @@ STAGE PLANS: Map Operator Tree: TableScan alias: decimal_vgby - Statistics: Num rows: 559 Data size: 127658 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 12288 Data size: 2165060 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: cint (type: int), cdecimal1 (type: decimal(20,10)), cdecimal2 (type: decimal(23,14)) outputColumnNames: cint, cdecimal1, cdecimal2 - Statistics: Num rows: 559 Data size: 127658 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 12288 Data size: 2165060 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(cdecimal1), max(cdecimal1), min(cdecimal1), sum(cdecimal1), avg(cdecimal1), stddev_pop(cdecimal1), stddev_samp(cdecimal1), count(cdecimal2), max(cdecimal2), min(cdecimal2), sum(cdecimal2), avg(cdecimal2), stddev_pop(cdecimal2), stddev_samp(cdecimal2), count() keys: cint (type: int) mode: hash outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15 - Statistics: Num rows: 559 Data size: 127658 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 12288 Data size: 2165060 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: 559 Data size: 127658 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 12288 Data size: 2165060 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint), _col2 (type: decimal(20,10)), _col3 (type: decimal(20,10)), _col4 (type: decimal(30,10)), _col5 (type: struct), _col6 (type: struct), _col7 (type: struct), _col8 (type: bigint), _col9 (type: decimal(23,14)), _col10 (type: decimal(23,14)), _col11 (type: decimal(33,14)), _col12 (type: struct), _col13 (type: struct), _col14 (type: struct), _col15 (type: bigint) Execution mode: vectorized Reducer 2 @@ -178,17 +177,17 @@ STAGE PLANS: keys: KEY._col0 (type: int) mode: mergepartial outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15 - Statistics: Num rows: 279 Data size: 63714 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 6144 Data size: 1082530 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (_col15 > 1) (type: boolean) - Statistics: Num rows: 93 Data size: 21238 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2048 Data size: 360843 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: decimal(20,10)), _col3 (type: decimal(20,10)), _col4 (type: decimal(30,10)), _col5 (type: decimal(24,14)), _col6 (type: double), _col7 (type: double), _col8 (type: bigint), _col9 (type: decimal(23,14)), _col10 (type: decimal(23,14)), _col11 (type: decimal(33,14)), _col12 (type: decimal(27,18)), _col13 (type: double), _col14 (type: double) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14 - Statistics: Num rows: 93 Data size: 21238 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2048 Data size: 360843 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 93 Data size: 21238 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 2048 Data size: 360843 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_13.q.out ql/src/test/results/clientpositive/spark/vectorization_13.q.out index 5be2c84..8e0c7bc 100644 --- ql/src/test/results/clientpositive/spark/vectorization_13.q.out +++ ql/src/test/results/clientpositive/spark/vectorization_13.q.out @@ -1,3 +1,132 @@ +PREHOOK: query: EXPLAIN +SELECT cboolean1, + ctinyint, + ctimestamp1, + cfloat, + cstring1, + (-(ctinyint)), + MAX(ctinyint), + ((-(ctinyint)) + MAX(ctinyint)), + SUM(cfloat), + (SUM(cfloat) * ((-(ctinyint)) + MAX(ctinyint))), + (-(SUM(cfloat))), + (79.553 * cfloat), + STDDEV_POP(cfloat), + (-(SUM(cfloat))), + STDDEV_POP(ctinyint), + (((-(ctinyint)) + MAX(ctinyint)) - 10.175), + (-((-(SUM(cfloat))))), + (-26.28 / (-((-(SUM(cfloat)))))), + MAX(cfloat), + ((SUM(cfloat) * ((-(ctinyint)) + MAX(ctinyint))) / ctinyint), + MIN(ctinyint) +FROM alltypesorc +WHERE (((cfloat < 3569) + AND ((10.175 >= cdouble) + AND (cboolean1 != 1))) + OR ((ctimestamp1 > 11) + AND ((ctimestamp2 != 12) + AND (ctinyint < 9763215.5639)))) +GROUP BY cboolean1, ctinyint, ctimestamp1, cfloat, cstring1 +LIMIT 40 +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT cboolean1, + ctinyint, + ctimestamp1, + cfloat, + cstring1, + (-(ctinyint)), + MAX(ctinyint), + ((-(ctinyint)) + MAX(ctinyint)), + SUM(cfloat), + (SUM(cfloat) * ((-(ctinyint)) + MAX(ctinyint))), + (-(SUM(cfloat))), + (79.553 * cfloat), + STDDEV_POP(cfloat), + (-(SUM(cfloat))), + STDDEV_POP(ctinyint), + (((-(ctinyint)) + MAX(ctinyint)) - 10.175), + (-((-(SUM(cfloat))))), + (-26.28 / (-((-(SUM(cfloat)))))), + MAX(cfloat), + ((SUM(cfloat) * ((-(ctinyint)) + MAX(ctinyint))) / ctinyint), + MIN(ctinyint) +FROM alltypesorc +WHERE (((cfloat < 3569) + AND ((10.175 >= cdouble) + AND (cboolean1 != 1))) + OR ((ctimestamp1 > 11) + AND ((ctimestamp2 != 12) + AND (ctinyint < 9763215.5639)))) +GROUP BY cboolean1, ctinyint, ctimestamp1, cfloat, cstring1 +LIMIT 40 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +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: alltypesorc + Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (((cfloat < 3569.0) and ((10.175 >= cdouble) and (cboolean1 <> 1))) or ((ctimestamp1 > 11) and ((ctimestamp2 <> 12) and (ctinyint < 9763215.5639)))) (type: boolean) + Statistics: Num rows: 2730 Data size: 83809 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: cboolean1 (type: boolean), ctinyint (type: tinyint), ctimestamp1 (type: timestamp), cfloat (type: float), cstring1 (type: string) + outputColumnNames: cboolean1, ctinyint, ctimestamp1, cfloat, cstring1 + Statistics: Num rows: 2730 Data size: 83809 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(ctinyint), sum(cfloat), stddev_pop(cfloat), stddev_pop(ctinyint), max(cfloat), min(ctinyint) + keys: cboolean1 (type: boolean), ctinyint (type: tinyint), ctimestamp1 (type: timestamp), cfloat (type: float), cstring1 (type: string) + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10 + Statistics: Num rows: 2730 Data size: 83809 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string) + sort order: +++++ + Map-reduce partition columns: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string) + Statistics: Num rows: 2730 Data size: 83809 Basic stats: COMPLETE Column stats: NONE + value expressions: _col5 (type: tinyint), _col6 (type: double), _col7 (type: struct), _col8 (type: struct), _col9 (type: float), _col10 (type: tinyint) + Execution mode: vectorized + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0), sum(VALUE._col1), stddev_pop(VALUE._col2), stddev_pop(VALUE._col3), max(VALUE._col4), min(VALUE._col5) + keys: KEY._col0 (type: boolean), KEY._col1 (type: tinyint), KEY._col2 (type: timestamp), KEY._col3 (type: float), KEY._col4 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10 + Statistics: Num rows: 1365 Data size: 41904 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string), (- _col1) (type: tinyint), _col5 (type: tinyint), ((- _col1) + _col5) (type: tinyint), _col6 (type: double), (_col6 * ((- _col1) + _col5)) (type: double), (- _col6) (type: double), (79.553 * _col3) (type: double), _col7 (type: double), (- _col6) (type: double), _col8 (type: double), (((- _col1) + _col5) - 10.175) (type: double), (- (- _col6)) (type: double), (-26.28 / (- (- _col6))) (type: double), _col9 (type: float), ((_col6 * ((- _col1) + _col5)) / _col1) (type: double), _col10 (type: tinyint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20 + Statistics: Num rows: 1365 Data size: 41904 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 40 + Statistics: Num rows: 40 Data size: 1200 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 40 Data size: 1200 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: 40 + Processor Tree: + ListSink + PREHOOK: query: SELECT cboolean1, ctinyint, ctimestamp1, @@ -23,10 +152,11 @@ FROM alltypesorc WHERE (((cfloat < 3569) AND ((10.175 >= cdouble) AND (cboolean1 != 1))) - OR ((ctimestamp1 > -29071) - AND ((ctimestamp2 != -29071) + OR ((ctimestamp1 > 11) + AND ((ctimestamp2 != 12) AND (ctinyint < 9763215.5639)))) GROUP BY cboolean1, ctinyint, ctimestamp1, cfloat, cstring1 +LIMIT 40 PREHOOK: type: QUERY PREHOOK: Input: default@alltypesorc #### A masked pattern was here #### @@ -55,25 +185,265 @@ FROM alltypesorc WHERE (((cfloat < 3569) AND ((10.175 >= cdouble) AND (cboolean1 != 1))) - OR ((ctimestamp1 > -29071) - AND ((ctimestamp2 != -29071) + OR ((ctimestamp1 > 11) + AND ((ctimestamp2 != 12) AND (ctinyint < 9763215.5639)))) GROUP BY cboolean1, ctinyint, ctimestamp1, cfloat, cstring1 +LIMIT 40 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@alltypesorc +#### A masked pattern was here #### +NULL -25 1969-12-31 16:00:13.374 -25.0 NULL 25 -25 0 -25.0 -0.0 25.0 -1988.8249999999998 0.0 25.0 0.0 -10.175 -25.0 1.0512000000000001 -25.0 0.0 -25 +NULL 62 1969-12-31 16:00:13.524 62.0 NULL -62 62 0 62.0 0.0 -62.0 4932.286 0.0 -62.0 0.0 -10.175 62.0 -0.4238709677419355 62.0 0.0 62 +NULL -38 1969-12-31 16:00:14.548 -38.0 NULL 38 -38 0 -38.0 -0.0 38.0 -3023.014 0.0 38.0 0.0 -10.175 -38.0 0.6915789473684211 -38.0 0.0 -38 +true 26 1969-12-31 16:00:14.341 26.0 1cGVWH7n1QU -26 26 0 26.0 0.0 -26.0 2068.3779999999997 0.0 -26.0 0.0 -10.175 26.0 -1.0107692307692309 26.0 0.0 26 +NULL -62 1969-12-31 16:00:12.388 -62.0 NULL 62 -62 0 -62.0 -0.0 62.0 -4932.286 0.0 62.0 0.0 -10.175 -62.0 0.4238709677419355 -62.0 0.0 -62 +NULL 28 1969-12-31 16:00:12.148 28.0 NULL -28 28 0 28.0 0.0 -28.0 2227.484 0.0 -28.0 0.0 -10.175 28.0 -0.9385714285714286 28.0 0.0 28 +true 49 1969-12-31 16:00:12.273 49.0 821UdmGbkEf4j -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 +true 13 1969-12-31 16:00:12.554 13.0 cvLH6Eat2yFsyy7p -13 13 0 13.0 0.0 -13.0 1034.1889999999999 0.0 -13.0 0.0 -10.175 13.0 -2.0215384615384617 13.0 0.0 13 +NULL 41 1969-12-31 16:00:14.017 41.0 NULL -41 41 0 41.0 0.0 -41.0 3261.673 0.0 -41.0 0.0 -10.175 41.0 -0.6409756097560976 41.0 0.0 41 +NULL -62 1969-12-31 16:00:14.247 -62.0 NULL 62 -62 0 -62.0 -0.0 62.0 -4932.286 0.0 62.0 0.0 -10.175 -62.0 0.4238709677419355 -62.0 0.0 -62 +NULL -49 1969-12-31 16:00:11.627 -49.0 NULL 49 -49 0 -49.0 -0.0 49.0 -3898.0969999999998 0.0 49.0 0.0 -10.175 -49.0 0.5363265306122449 -49.0 0.0 -49 +NULL 40 1969-12-31 16:00:14.977 40.0 NULL -40 40 0 40.0 0.0 -40.0 3182.12 0.0 -40.0 0.0 -10.175 40.0 -0.657 40.0 0.0 40 +NULL 13 1969-12-31 16:00:14.548 13.0 NULL -13 13 0 13.0 0.0 -13.0 1034.1889999999999 0.0 -13.0 0.0 -10.175 13.0 -2.0215384615384617 13.0 0.0 13 +NULL -2 1969-12-31 16:00:14.391 -2.0 NULL 2 -2 0 -2.0 -0.0 2.0 -159.106 0.0 2.0 0.0 -10.175 -2.0 13.14 -2.0 0.0 -2 +true -22 1969-12-31 16:00:14.318 -22.0 1cGVWH7n1QU 22 -22 0 -22.0 -0.0 22.0 -1750.166 0.0 22.0 0.0 -10.175 -22.0 1.1945454545454546 -22.0 0.0 -22 +true 34 1969-12-31 16:00:13.221 34.0 821UdmGbkEf4j -34 34 0 34.0 0.0 -34.0 2704.8019999999997 0.0 -34.0 0.0 -10.175 34.0 -0.7729411764705882 34.0 0.0 34 +NULL 18 1969-12-31 16:00:12.437 18.0 NULL -18 18 0 18.0 0.0 -18.0 1431.954 0.0 -18.0 0.0 -10.175 18.0 -1.46 18.0 0.0 18 +NULL 62 1969-12-31 16:00:14.982 62.0 NULL -62 62 0 62.0 0.0 -62.0 4932.286 0.0 -62.0 0.0 -10.175 62.0 -0.4238709677419355 62.0 0.0 62 +true 38 1969-12-31 16:00:14.239 38.0 821UdmGbkEf4j -38 38 0 38.0 0.0 -38.0 3023.014 0.0 -38.0 0.0 -10.175 38.0 -0.6915789473684211 38.0 0.0 38 +true 61 1969-12-31 16:00:15.673 61.0 1cGVWH7n1QU -61 61 0 61.0 0.0 -61.0 4852.733 0.0 -61.0 0.0 -10.175 61.0 -0.4308196721311476 61.0 0.0 61 +true 9 1969-12-31 16:00:11.544 9.0 1cGVWH7n1QU -9 9 0 9.0 0.0 -9.0 715.977 0.0 -9.0 0.0 -10.175 9.0 -2.92 9.0 0.0 9 +NULL -51 1969-12-31 16:00:12.494 -51.0 NULL 51 -51 0 -51.0 -0.0 51.0 -4057.203 0.0 51.0 0.0 -10.175 -51.0 0.5152941176470588 -51.0 0.0 -51 +true 25 1969-12-31 16:00:13.338 25.0 cvLH6Eat2yFsyy7p -25 25 0 25.0 0.0 -25.0 1988.8249999999998 0.0 -25.0 0.0 -10.175 25.0 -1.0512000000000001 25.0 0.0 25 +true -14 1969-12-31 16:00:15.625 -14.0 cvLH6Eat2yFsyy7p 14 -14 0 -14.0 -0.0 14.0 -1113.742 0.0 14.0 0.0 -10.175 -14.0 1.8771428571428572 -14.0 0.0 -14 +true -34 1969-12-31 16:00:15.254 -34.0 821UdmGbkEf4j 34 -34 0 -34.0 -0.0 34.0 -2704.8019999999997 0.0 34.0 0.0 -10.175 -34.0 0.7729411764705882 -34.0 0.0 -34 +true -1 1969-12-31 16:00:16.189 -1.0 cvLH6Eat2yFsyy7p 1 -1 0 -1.0 -0.0 1.0 -79.553 0.0 1.0 0.0 -10.175 -1.0 26.28 -1.0 0.0 -1 +NULL -30 1969-12-31 16:00:12.935 -30.0 NULL 30 -30 0 -30.0 -0.0 30.0 -2386.59 0.0 30.0 0.0 -10.175 -30.0 0.876 -30.0 0.0 -30 +true -53 1969-12-31 16:00:11.977 -53.0 821UdmGbkEf4j 53 -53 0 -53.0 -0.0 53.0 -4216.309 0.0 53.0 0.0 -10.175 -53.0 0.4958490566037736 -53.0 0.0 -53 +true 6 1969-12-31 16:00:11.357 6.0 cvLH6Eat2yFsyy7p -6 6 0 6.0 0.0 -6.0 477.318 0.0 -6.0 0.0 -10.175 6.0 -4.38 6.0 0.0 6 +NULL 30 1969-12-31 16:00:11.876 30.0 NULL -30 30 0 30.0 0.0 -30.0 2386.59 0.0 -30.0 0.0 -10.175 30.0 -0.876 30.0 0.0 30 +NULL 48 1969-12-31 16:00:15.512 48.0 NULL -48 48 0 48.0 0.0 -48.0 3818.544 0.0 -48.0 0.0 -10.175 48.0 -0.5475 48.0 0.0 48 +NULL -21 1969-12-31 16:00:13.32 -21.0 NULL 21 -21 0 -21.0 -0.0 21.0 -1670.6129999999998 0.0 21.0 0.0 -10.175 -21.0 1.2514285714285716 -21.0 0.0 -21 +NULL -42 1969-12-31 16:00:16.191 -42.0 NULL 42 -42 0 -42.0 -0.0 42.0 -3341.2259999999997 0.0 42.0 0.0 -10.175 -42.0 0.6257142857142858 -42.0 0.0 -42 +NULL 0 1969-12-31 16:00:11.089 0.0 NULL 0 0 0 0.0 0.0 -0.0 0.0 0.0 -0.0 0.0 -10.175 0.0 NULL 0.0 NULL 0 +true -58 1969-12-31 16:00:13.209 -58.0 cvLH6Eat2yFsyy7p 58 -58 0 -58.0 -0.0 58.0 -4614.074 0.0 58.0 0.0 -10.175 -58.0 0.4531034482758621 -58.0 0.0 -58 +NULL 60 1969-12-31 16:00:11.003 60.0 NULL -60 60 0 60.0 0.0 -60.0 4773.18 0.0 -60.0 0.0 -10.175 60.0 -0.438 60.0 0.0 60 +true 23 1969-12-31 16:00:11.343 23.0 cvLH6Eat2yFsyy7p -23 23 0 23.0 0.0 -23.0 1829.719 0.0 -23.0 0.0 -10.175 23.0 -1.142608695652174 23.0 0.0 23 +NULL -27 1969-12-31 16:00:14.065 -27.0 NULL 27 -27 0 -27.0 -0.0 27.0 -2147.931 0.0 27.0 0.0 -10.175 -27.0 0.9733333333333334 -27.0 0.0 -27 +true 57 1969-12-31 16:00:13.71 57.0 cvLH6Eat2yFsyy7p -57 57 0 57.0 0.0 -57.0 4534.521 0.0 -57.0 0.0 -10.175 57.0 -0.4610526315789474 57.0 0.0 57 +true -59 1969-12-31 16:00:12.227 -59.0 cvLH6Eat2yFsyy7p 59 -59 0 -59.0 -0.0 59.0 -4693.6269999999995 0.0 59.0 0.0 -10.175 -59.0 0.44542372881355935 -59.0 0.0 -59 +PREHOOK: query: -- double compare timestamp +EXPLAIN +SELECT cboolean1, + ctinyint, + ctimestamp1, + cfloat, + cstring1, + (-(ctinyint)), + MAX(ctinyint), + ((-(ctinyint)) + MAX(ctinyint)), + SUM(cfloat), + (SUM(cfloat) * ((-(ctinyint)) + MAX(ctinyint))), + (-(SUM(cfloat))), + (79.553 * cfloat), + STDDEV_POP(cfloat), + (-(SUM(cfloat))), + STDDEV_POP(ctinyint), + (((-(ctinyint)) + MAX(ctinyint)) - 10.175), + (-((-(SUM(cfloat))))), + (-26.28 / (-((-(SUM(cfloat)))))), + MAX(cfloat), + ((SUM(cfloat) * ((-(ctinyint)) + MAX(ctinyint))) / ctinyint), + MIN(ctinyint) +FROM alltypesorc +WHERE (((cfloat < 3569) + AND ((10.175 >= cdouble) + AND (cboolean1 != 1))) + OR ((ctimestamp1 > -1.388) + AND ((ctimestamp2 != -1.3359999999999999) + AND (ctinyint < 9763215.5639)))) +GROUP BY cboolean1, ctinyint, ctimestamp1, cfloat, cstring1 +LIMIT 40 +PREHOOK: type: QUERY +POSTHOOK: query: -- double compare timestamp +EXPLAIN +SELECT cboolean1, + ctinyint, + ctimestamp1, + cfloat, + cstring1, + (-(ctinyint)), + MAX(ctinyint), + ((-(ctinyint)) + MAX(ctinyint)), + SUM(cfloat), + (SUM(cfloat) * ((-(ctinyint)) + MAX(ctinyint))), + (-(SUM(cfloat))), + (79.553 * cfloat), + STDDEV_POP(cfloat), + (-(SUM(cfloat))), + STDDEV_POP(ctinyint), + (((-(ctinyint)) + MAX(ctinyint)) - 10.175), + (-((-(SUM(cfloat))))), + (-26.28 / (-((-(SUM(cfloat)))))), + MAX(cfloat), + ((SUM(cfloat) * ((-(ctinyint)) + MAX(ctinyint))) / ctinyint), + MIN(ctinyint) +FROM alltypesorc +WHERE (((cfloat < 3569) + AND ((10.175 >= cdouble) + AND (cboolean1 != 1))) + OR ((ctimestamp1 > -1.388) + AND ((ctimestamp2 != -1.3359999999999999) + AND (ctinyint < 9763215.5639)))) +GROUP BY cboolean1, ctinyint, ctimestamp1, cfloat, cstring1 +LIMIT 40 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +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: alltypesorc + Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (((cfloat < 3569.0) and ((10.175 >= cdouble) and (cboolean1 <> 1))) or ((ctimestamp1 > -1.388) and ((ctimestamp2 <> -1.3359999999999999) and (ctinyint < 9763215.5639)))) (type: boolean) + Statistics: Num rows: 2730 Data size: 83809 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: cboolean1 (type: boolean), ctinyint (type: tinyint), ctimestamp1 (type: timestamp), cfloat (type: float), cstring1 (type: string) + outputColumnNames: cboolean1, ctinyint, ctimestamp1, cfloat, cstring1 + Statistics: Num rows: 2730 Data size: 83809 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(ctinyint), sum(cfloat), stddev_pop(cfloat), stddev_pop(ctinyint), max(cfloat), min(ctinyint) + keys: cboolean1 (type: boolean), ctinyint (type: tinyint), ctimestamp1 (type: timestamp), cfloat (type: float), cstring1 (type: string) + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10 + Statistics: Num rows: 2730 Data size: 83809 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string) + sort order: +++++ + Map-reduce partition columns: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string) + Statistics: Num rows: 2730 Data size: 83809 Basic stats: COMPLETE Column stats: NONE + value expressions: _col5 (type: tinyint), _col6 (type: double), _col7 (type: struct), _col8 (type: struct), _col9 (type: float), _col10 (type: tinyint) + Execution mode: vectorized + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0), sum(VALUE._col1), stddev_pop(VALUE._col2), stddev_pop(VALUE._col3), max(VALUE._col4), min(VALUE._col5) + keys: KEY._col0 (type: boolean), KEY._col1 (type: tinyint), KEY._col2 (type: timestamp), KEY._col3 (type: float), KEY._col4 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10 + Statistics: Num rows: 1365 Data size: 41904 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: boolean), _col1 (type: tinyint), _col2 (type: timestamp), _col3 (type: float), _col4 (type: string), (- _col1) (type: tinyint), _col5 (type: tinyint), ((- _col1) + _col5) (type: tinyint), _col6 (type: double), (_col6 * ((- _col1) + _col5)) (type: double), (- _col6) (type: double), (79.553 * _col3) (type: double), _col7 (type: double), (- _col6) (type: double), _col8 (type: double), (((- _col1) + _col5) - 10.175) (type: double), (- (- _col6)) (type: double), (-26.28 / (- (- _col6))) (type: double), _col9 (type: float), ((_col6 * ((- _col1) + _col5)) / _col1) (type: double), _col10 (type: tinyint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20 + Statistics: Num rows: 1365 Data size: 41904 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 40 + Statistics: Num rows: 40 Data size: 1200 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 40 Data size: 1200 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: 40 + Processor Tree: + ListSink + +PREHOOK: query: SELECT cboolean1, + ctinyint, + ctimestamp1, + cfloat, + cstring1, + (-(ctinyint)), + MAX(ctinyint), + ((-(ctinyint)) + MAX(ctinyint)), + SUM(cfloat), + (SUM(cfloat) * ((-(ctinyint)) + MAX(ctinyint))), + (-(SUM(cfloat))), + (79.553 * cfloat), + STDDEV_POP(cfloat), + (-(SUM(cfloat))), + STDDEV_POP(ctinyint), + (((-(ctinyint)) + MAX(ctinyint)) - 10.175), + (-((-(SUM(cfloat))))), + (-26.28 / (-((-(SUM(cfloat)))))), + MAX(cfloat), + ((SUM(cfloat) * ((-(ctinyint)) + MAX(ctinyint))) / ctinyint), + MIN(ctinyint) +FROM alltypesorc +WHERE (((cfloat < 3569) + AND ((10.175 >= cdouble) + AND (cboolean1 != 1))) + OR ((ctimestamp1 > -1.388) + AND ((ctimestamp2 != -1.3359999999999999) + AND (ctinyint < 9763215.5639)))) +GROUP BY cboolean1, ctinyint, ctimestamp1, cfloat, cstring1 +LIMIT 40 +PREHOOK: type: QUERY +PREHOOK: Input: default@alltypesorc +#### A masked pattern was here #### +POSTHOOK: query: SELECT cboolean1, + ctinyint, + ctimestamp1, + cfloat, + cstring1, + (-(ctinyint)), + MAX(ctinyint), + ((-(ctinyint)) + MAX(ctinyint)), + SUM(cfloat), + (SUM(cfloat) * ((-(ctinyint)) + MAX(ctinyint))), + (-(SUM(cfloat))), + (79.553 * cfloat), + STDDEV_POP(cfloat), + (-(SUM(cfloat))), + STDDEV_POP(ctinyint), + (((-(ctinyint)) + MAX(ctinyint)) - 10.175), + (-((-(SUM(cfloat))))), + (-26.28 / (-((-(SUM(cfloat)))))), + MAX(cfloat), + ((SUM(cfloat) * ((-(ctinyint)) + MAX(ctinyint))) / ctinyint), + MIN(ctinyint) +FROM alltypesorc +WHERE (((cfloat < 3569) + AND ((10.175 >= cdouble) + AND (cboolean1 != 1))) + OR ((ctimestamp1 > -1.388) + AND ((ctimestamp2 != -1.3359999999999999) + AND (ctinyint < 9763215.5639)))) +GROUP BY cboolean1, ctinyint, ctimestamp1, cfloat, cstring1 +LIMIT 40 POSTHOOK: type: QUERY POSTHOOK: Input: default@alltypesorc #### A masked pattern was here #### true -21 1969-12-31 16:00:00.826 -21.0 1cGVWH7n1QU 21 -21 0 -21.0 -0.0 21.0 -1670.6129999999998 0.0 21.0 0.0 -10.175 -21.0 1.2514285714285716 -21.0 0.0 -21 true 26 1969-12-31 16:00:14.341 26.0 1cGVWH7n1QU -26 26 0 26.0 0.0 -26.0 2068.3779999999997 0.0 -26.0 0.0 -10.175 26.0 -1.0107692307692309 26.0 0.0 26 NULL -62 1969-12-31 16:00:12.388 -62.0 NULL 62 -62 0 -62.0 -0.0 62.0 -4932.286 0.0 62.0 0.0 -10.175 -62.0 0.4238709677419355 -62.0 0.0 -62 -NULL 26 1969-12-31 16:00:04.699 26.0 NULL -26 26 0 26.0 0.0 -26.0 2068.3779999999997 0.0 -26.0 0.0 -10.175 26.0 -1.0107692307692309 26.0 0.0 26 +NULL -16 1969-12-31 15:59:58.727 -16.0 NULL 16 -16 0 -16.0 -0.0 16.0 -1272.848 0.0 16.0 0.0 -10.175 -16.0 1.6425 -16.0 0.0 -16 NULL -24 1969-12-31 16:00:07.302 -24.0 NULL 24 -24 0 -24.0 -0.0 24.0 -1909.272 0.0 24.0 0.0 -10.175 -24.0 1.095 -24.0 0.0 -24 true -10 1969-12-31 16:00:05.636 -10.0 1cGVWH7n1QU 10 -10 0 -10.0 -0.0 10.0 -795.53 0.0 10.0 0.0 -10.175 -10.0 2.628 -10.0 0.0 -10 NULL 41 1969-12-31 16:00:14.017 41.0 NULL -41 41 0 41.0 0.0 -41.0 3261.673 0.0 -41.0 0.0 -10.175 41.0 -0.6409756097560976 41.0 0.0 41 +NULL 26 1969-12-31 16:00:04.699 26.0 NULL -26 26 0 26.0 0.0 -26.0 2068.3779999999997 0.0 -26.0 0.0 -10.175 26.0 -1.0107692307692309 26.0 0.0 26 NULL 18 1969-12-31 16:00:02.295 18.0 NULL -18 18 0 18.0 0.0 -18.0 1431.954 0.0 -18.0 0.0 -10.175 18.0 -1.46 18.0 0.0 18 true 61 1969-12-31 16:00:05.394 61.0 cvLH6Eat2yFsyy7p -61 61 0 61.0 0.0 -61.0 4852.733 0.0 -61.0 0.0 -10.175 61.0 -0.4308196721311476 61.0 0.0 61 true 9 1969-12-31 16:00:11.544 9.0 1cGVWH7n1QU -9 9 0 9.0 0.0 -9.0 715.977 0.0 -9.0 0.0 -10.175 9.0 -2.92 9.0 0.0 9 -NULL -41 1969-12-31 16:00:00.096 -41.0 NULL 41 -41 0 -41.0 -0.0 41.0 -3261.673 0.0 41.0 0.0 -10.175 -41.0 0.6409756097560976 -41.0 0.0 -41 +true 55 1969-12-31 15:59:58.962 55.0 cvLH6Eat2yFsyy7p -55 55 0 55.0 0.0 -55.0 4375.415 0.0 -55.0 0.0 -10.175 55.0 -0.47781818181818186 55.0 0.0 55 true 52 1969-12-31 16:00:07.428 52.0 1cGVWH7n1QU -52 52 0 52.0 0.0 -52.0 4136.755999999999 0.0 -52.0 0.0 -10.175 52.0 -0.5053846153846154 52.0 0.0 52 +NULL -41 1969-12-31 16:00:00.096 -41.0 NULL 41 -41 0 -41.0 -0.0 41.0 -3261.673 0.0 41.0 0.0 -10.175 -41.0 0.6409756097560976 -41.0 0.0 -41 NULL -51 1969-12-31 16:00:12.494 -51.0 NULL 51 -51 0 -51.0 -0.0 51.0 -4057.203 0.0 51.0 0.0 -10.175 -51.0 0.5152941176470588 -51.0 0.0 -51 NULL -18 1969-12-31 16:00:06.015 -18.0 NULL 18 -18 0 -18.0 -0.0 18.0 -1431.954 0.0 18.0 0.0 -10.175 -18.0 1.46 -18.0 0.0 -18 true -34 1969-12-31 16:00:15.254 -34.0 821UdmGbkEf4j 34 -34 0 -34.0 -0.0 34.0 -2704.8019999999997 0.0 34.0 0.0 -10.175 -34.0 0.7729411764705882 -34.0 0.0 -34 @@ -100,3177 +470,3 @@ NULL 42 1969-12-31 16:00:16.251 42.0 NULL -42 42 0 42.0 0.0 -42.0 3341.225999999 NULL 11 1969-12-31 16:00:15.338 11.0 NULL -11 11 0 11.0 0.0 -11.0 875.083 0.0 -11.0 0.0 -10.175 11.0 -2.389090909090909 11.0 0.0 11 NULL 39 1969-12-31 16:00:14.32 39.0 NULL -39 39 0 39.0 0.0 -39.0 3102.567 0.0 -39.0 0.0 -10.175 39.0 -0.6738461538461539 39.0 0.0 39 NULL 9 1969-12-31 16:00:00.945 9.0 NULL -9 9 0 9.0 0.0 -9.0 715.977 0.0 -9.0 0.0 -10.175 9.0 -2.92 9.0 0.0 9 -NULL 44 1969-12-31 16:00:08.171 44.0 NULL -44 44 0 44.0 0.0 -44.0 3500.332 0.0 -44.0 0.0 -10.175 44.0 -0.5972727272727273 44.0 0.0 44 -NULL 11 1969-12-31 16:00:07.204 11.0 NULL -11 11 0 11.0 0.0 -11.0 875.083 0.0 -11.0 0.0 -10.175 11.0 -2.389090909090909 11.0 0.0 11 -true 16 1969-12-31 16:00:06.813 16.0 1cGVWH7n1QU -16 16 0 16.0 0.0 -16.0 1272.848 0.0 -16.0 0.0 -10.175 16.0 -1.6425 16.0 0.0 16 -true 5 1969-12-31 16:00:13.612 5.0 cvLH6Eat2yFsyy7p -5 5 0 5.0 0.0 -5.0 397.765 0.0 -5.0 0.0 -10.175 5.0 -5.256 5.0 0.0 5 -NULL -23 1969-12-31 16:00:06.243 -23.0 NULL 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -NULL -35 1969-12-31 16:00:12.959 -35.0 NULL 35 -35 0 -35.0 -0.0 35.0 -2784.355 0.0 35.0 0.0 -10.175 -35.0 0.7508571428571429 -35.0 0.0 -35 -NULL 22 1969-12-31 16:00:15.715 22.0 NULL -22 22 0 22.0 0.0 -22.0 1750.166 0.0 -22.0 0.0 -10.175 22.0 -1.1945454545454546 22.0 0.0 22 -NULL -3 1969-12-31 16:00:00.02 -3.0 NULL 3 -3 0 -3.0 -0.0 3.0 -238.659 0.0 3.0 0.0 -10.175 -3.0 8.76 -3.0 0.0 -3 -true 33 1969-12-31 16:00:10.724 33.0 1cGVWH7n1QU -33 33 0 33.0 0.0 -33.0 2625.249 0.0 -33.0 0.0 -10.175 33.0 -0.7963636363636364 33.0 0.0 33 -NULL 56 1969-12-31 16:00:00.909 56.0 NULL -56 56 0 56.0 0.0 -56.0 4454.968 0.0 -56.0 0.0 -10.175 56.0 -0.4692857142857143 56.0 0.0 56 -NULL -25 1969-12-31 16:00:11.469 -25.0 NULL 25 -25 0 -25.0 -0.0 25.0 -1988.8249999999998 0.0 25.0 0.0 -10.175 -25.0 1.0512000000000001 -25.0 0.0 -25 -true -1 1969-12-31 16:00:03.106 -1.389 1cGVWH7n1QU 1 -1 0 -1.3890000581741333 -0.0 1.3890000581741333 -110.49912162792683 0.0 1.3890000581741333 0.0 -10.175 -1.3890000581741333 18.920085600677048 -1.389 0.0 -1 -NULL 42 1969-12-31 16:00:10.894 42.0 NULL -42 42 0 42.0 0.0 -42.0 3341.2259999999997 0.0 -42.0 0.0 -10.175 42.0 -0.6257142857142858 42.0 0.0 42 -true 42 1969-12-31 16:00:09.782 42.0 821UdmGbkEf4j -42 42 0 42.0 0.0 -42.0 3341.2259999999997 0.0 -42.0 0.0 -10.175 42.0 -0.6257142857142858 42.0 0.0 42 -NULL -49 1969-12-31 16:00:02.05 -49.0 NULL 49 -49 0 -49.0 -0.0 49.0 -3898.0969999999998 0.0 49.0 0.0 -10.175 -49.0 0.5363265306122449 -49.0 0.0 -49 -true -18 1969-12-31 16:00:11.022 -18.0 821UdmGbkEf4j 18 -18 0 -18.0 -0.0 18.0 -1431.954 0.0 18.0 0.0 -10.175 -18.0 1.46 -18.0 0.0 -18 -true -23 1969-12-31 16:00:05.259 -23.0 cvLH6Eat2yFsyy7p 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -true -59 1969-12-31 16:00:15.242 -59.0 821UdmGbkEf4j 59 -59 0 -59.0 -0.0 59.0 -4693.6269999999995 0.0 59.0 0.0 -10.175 -59.0 0.44542372881355935 -59.0 0.0 -59 -true 42 1969-12-31 16:00:11.391 42.0 cvLH6Eat2yFsyy7p -42 42 0 42.0 0.0 -42.0 3341.2259999999997 0.0 -42.0 0.0 -10.175 42.0 -0.6257142857142858 42.0 0.0 42 -true -13 1969-12-31 16:00:07.375 -13.0 821UdmGbkEf4j 13 -13 0 -13.0 -0.0 13.0 -1034.1889999999999 0.0 13.0 0.0 -10.175 -13.0 2.0215384615384617 -13.0 0.0 -13 -true 20 1969-12-31 16:00:01.943 20.0 1cGVWH7n1QU -20 20 0 20.0 0.0 -20.0 1591.06 0.0 -20.0 0.0 -10.175 20.0 -1.314 20.0 0.0 20 -NULL 34 1969-12-31 16:00:14.909 34.0 NULL -34 34 0 34.0 0.0 -34.0 2704.8019999999997 0.0 -34.0 0.0 -10.175 34.0 -0.7729411764705882 34.0 0.0 34 -true -4 1969-12-31 16:00:05.323 -4.0 821UdmGbkEf4j 4 -4 0 -4.0 -0.0 4.0 -318.212 0.0 4.0 0.0 -10.175 -4.0 6.57 -4.0 0.0 -4 -true 11 1969-12-31 16:00:02.389 11.0 cvLH6Eat2yFsyy7p -11 11 0 11.0 0.0 -11.0 875.083 0.0 -11.0 0.0 -10.175 11.0 -2.389090909090909 11.0 0.0 11 -NULL -34 1969-12-31 16:00:11.611 -34.0 NULL 34 -34 0 -34.0 -0.0 34.0 -2704.8019999999997 0.0 34.0 0.0 -10.175 -34.0 0.7729411764705882 -34.0 0.0 -34 -NULL 54 1969-12-31 16:00:01.841 54.0 NULL -54 54 0 54.0 0.0 -54.0 4295.862 0.0 -54.0 0.0 -10.175 54.0 -0.4866666666666667 54.0 0.0 54 -NULL -19 1969-12-31 16:00:05.914 -19.0 NULL 19 -19 0 -19.0 -0.0 19.0 -1511.507 0.0 19.0 0.0 -10.175 -19.0 1.3831578947368421 -19.0 0.0 -19 -NULL -20 1969-12-31 16:00:13.066 -20.0 NULL 20 -20 0 -20.0 -0.0 20.0 -1591.06 0.0 20.0 0.0 -10.175 -20.0 1.314 -20.0 0.0 -20 -true 37 1969-12-31 16:00:15.562 37.0 1cGVWH7n1QU -37 37 0 37.0 0.0 -37.0 2943.461 0.0 -37.0 0.0 -10.175 37.0 -0.7102702702702703 37.0 0.0 37 -NULL 24 1969-12-31 16:00:15.061 24.0 NULL -24 24 0 24.0 0.0 -24.0 1909.272 0.0 -24.0 0.0 -10.175 24.0 -1.095 24.0 0.0 24 -NULL 49 1969-12-31 16:00:12.104 49.0 NULL -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 -true -54 1969-12-31 16:00:01.822 -54.0 cvLH6Eat2yFsyy7p 54 -54 0 -54.0 -0.0 54.0 -4295.862 0.0 54.0 0.0 -10.175 -54.0 0.4866666666666667 -54.0 0.0 -54 -true -45 1969-12-31 16:00:00.01 -45.0 cvLH6Eat2yFsyy7p 45 -45 0 -45.0 -0.0 45.0 -3579.8849999999998 0.0 45.0 0.0 -10.175 -45.0 0.5840000000000001 -45.0 0.0 -45 -true 50 1969-12-31 16:00:03.971 50.0 821UdmGbkEf4j -50 50 0 50.0 0.0 -50.0 3977.6499999999996 0.0 -50.0 0.0 -10.175 50.0 -0.5256000000000001 50.0 0.0 50 -NULL -3 1969-12-31 16:00:11.059 -3.0 NULL 3 -3 0 -3.0 -0.0 3.0 -238.659 0.0 3.0 0.0 -10.175 -3.0 8.76 -3.0 0.0 -3 -NULL 61 1969-12-31 16:00:15.143 61.0 NULL -61 61 0 61.0 0.0 -61.0 4852.733 0.0 -61.0 0.0 -10.175 61.0 -0.4308196721311476 61.0 0.0 61 -NULL 39 1969-12-31 16:00:00.395 39.0 NULL -39 39 0 39.0 0.0 -39.0 3102.567 0.0 -39.0 0.0 -10.175 39.0 -0.6738461538461539 39.0 0.0 39 -true 26 1969-12-31 16:00:08.896 26.0 821UdmGbkEf4j -26 26 0 26.0 0.0 -26.0 2068.3779999999997 0.0 -26.0 0.0 -10.175 26.0 -1.0107692307692309 26.0 0.0 26 -NULL 30 1969-12-31 16:00:04.199 30.0 NULL -30 30 0 30.0 0.0 -30.0 2386.59 0.0 -30.0 0.0 -10.175 30.0 -0.876 30.0 0.0 30 -true -20 1969-12-31 16:00:00.063 -20.0 cvLH6Eat2yFsyy7p 20 -20 0 -20.0 -0.0 20.0 -1591.06 0.0 20.0 0.0 -10.175 -20.0 1.314 -20.0 0.0 -20 -NULL 47 1969-12-31 16:00:05.344 47.0 NULL -47 47 0 47.0 0.0 -47.0 3738.991 0.0 -47.0 0.0 -10.175 47.0 -0.5591489361702128 47.0 0.0 47 -NULL 44 1969-12-31 16:00:00.206 44.0 NULL -44 44 0 44.0 0.0 -44.0 3500.332 0.0 -44.0 0.0 -10.175 44.0 -0.5972727272727273 44.0 0.0 44 -true -60 1969-12-31 16:00:05.661 -60.0 821UdmGbkEf4j 60 -60 0 -60.0 -0.0 60.0 -4773.18 0.0 60.0 0.0 -10.175 -60.0 0.438 -60.0 0.0 -60 -NULL 35 1969-12-31 16:00:02.494 35.0 NULL -35 35 0 35.0 0.0 -35.0 2784.355 0.0 -35.0 0.0 -10.175 35.0 -0.7508571428571429 35.0 0.0 35 -true -59 1969-12-31 16:00:07.905 -59.0 1cGVWH7n1QU 59 -59 0 -59.0 -0.0 59.0 -4693.6269999999995 0.0 59.0 0.0 -10.175 -59.0 0.44542372881355935 -59.0 0.0 -59 -NULL 13 1969-12-31 16:00:15.956 13.0 NULL -13 13 0 13.0 0.0 -13.0 1034.1889999999999 0.0 -13.0 0.0 -10.175 13.0 -2.0215384615384617 13.0 0.0 13 -true 13 1969-12-31 16:00:06.453 13.0 cvLH6Eat2yFsyy7p -13 13 0 13.0 0.0 -13.0 1034.1889999999999 0.0 -13.0 0.0 -10.175 13.0 -2.0215384615384617 13.0 0.0 13 -NULL -61 1969-12-31 16:00:14.192 -61.0 NULL 61 -61 0 -61.0 -0.0 61.0 -4852.733 0.0 61.0 0.0 -10.175 -61.0 0.4308196721311476 -61.0 0.0 -61 -true -38 1969-12-31 16:00:01.845 -38.0 cvLH6Eat2yFsyy7p 38 -38 0 -38.0 -0.0 38.0 -3023.014 0.0 38.0 0.0 -10.175 -38.0 0.6915789473684211 -38.0 0.0 -38 -true 35 1969-12-31 16:00:02.583 35.0 1cGVWH7n1QU -35 35 0 35.0 0.0 -35.0 2784.355 0.0 -35.0 0.0 -10.175 35.0 -0.7508571428571429 35.0 0.0 35 -NULL -25 1969-12-31 16:00:15.965 -25.0 NULL 25 -25 0 -25.0 -0.0 25.0 -1988.8249999999998 0.0 25.0 0.0 -10.175 -25.0 1.0512000000000001 -25.0 0.0 -25 -NULL -8 1969-12-31 16:00:03.809 -8.0 NULL 8 -8 0 -8.0 -0.0 8.0 -636.424 0.0 8.0 0.0 -10.175 -8.0 3.285 -8.0 0.0 -8 -NULL -57 1969-12-31 16:00:09.925 -57.0 NULL 57 -57 0 -57.0 -0.0 57.0 -4534.521 0.0 57.0 0.0 -10.175 -57.0 0.4610526315789474 -57.0 0.0 -57 -NULL 44 1969-12-31 16:00:07.731 44.0 NULL -44 44 0 44.0 0.0 -44.0 3500.332 0.0 -44.0 0.0 -10.175 44.0 -0.5972727272727273 44.0 0.0 44 -true -46 1969-12-31 16:00:12.348 -46.0 821UdmGbkEf4j 46 -46 0 -46.0 -0.0 46.0 -3659.438 0.0 46.0 0.0 -10.175 -46.0 0.571304347826087 -46.0 0.0 -46 -NULL -5 1969-12-31 16:00:03.52 -5.0 NULL 5 -5 0 -5.0 -0.0 5.0 -397.765 0.0 5.0 0.0 -10.175 -5.0 5.256 -5.0 0.0 -5 -true -59 1969-12-31 16:00:11.231 -59.0 821UdmGbkEf4j 59 -59 0 -59.0 -0.0 59.0 -4693.6269999999995 0.0 59.0 0.0 -10.175 -59.0 0.44542372881355935 -59.0 0.0 -59 -true -15 1969-12-31 16:00:11.513 -15.0 1cGVWH7n1QU 15 -15 0 -15.0 -0.0 15.0 -1193.295 0.0 15.0 0.0 -10.175 -15.0 1.752 -15.0 0.0 -15 -NULL 12 1969-12-31 16:00:15.27 12.0 NULL -12 12 0 12.0 0.0 -12.0 954.636 0.0 -12.0 0.0 -10.175 12.0 -2.19 12.0 0.0 12 -NULL 12 1969-12-31 16:00:00.526 12.0 NULL -12 12 0 12.0 0.0 -12.0 954.636 0.0 -12.0 0.0 -10.175 12.0 -2.19 12.0 0.0 12 -true -19 1969-12-31 16:00:01.339 -19.0 1cGVWH7n1QU 19 -19 0 -19.0 -0.0 19.0 -1511.507 0.0 19.0 0.0 -10.175 -19.0 1.3831578947368421 -19.0 0.0 -19 -true 4 1969-12-31 16:00:06.663 4.0 cvLH6Eat2yFsyy7p -4 4 0 4.0 0.0 -4.0 318.212 0.0 -4.0 0.0 -10.175 4.0 -6.57 4.0 0.0 4 -true -62 1969-12-31 16:00:00.659 -62.0 821UdmGbkEf4j 62 -62 0 -62.0 -0.0 62.0 -4932.286 0.0 62.0 0.0 -10.175 -62.0 0.4238709677419355 -62.0 0.0 -62 -NULL -48 1969-12-31 16:00:06.363 -48.0 NULL 48 -48 0 -48.0 -0.0 48.0 -3818.544 0.0 48.0 0.0 -10.175 -48.0 0.5475 -48.0 0.0 -48 -NULL -48 1969-12-31 16:00:01.813 -48.0 NULL 48 -48 0 -48.0 -0.0 48.0 -3818.544 0.0 48.0 0.0 -10.175 -48.0 0.5475 -48.0 0.0 -48 -NULL -59 1969-12-31 16:00:05.927 -59.0 NULL 59 -59 0 -59.0 -0.0 59.0 -4693.6269999999995 0.0 59.0 0.0 -10.175 -59.0 0.44542372881355935 -59.0 0.0 -59 -NULL 44 1969-12-31 16:00:09.07 44.0 NULL -44 44 0 44.0 0.0 -44.0 3500.332 0.0 -44.0 0.0 -10.175 44.0 -0.5972727272727273 44.0 0.0 44 -NULL -19 1969-12-31 16:00:13.495 -19.0 NULL 19 -19 0 -19.0 -0.0 19.0 -1511.507 0.0 19.0 0.0 -10.175 -19.0 1.3831578947368421 -19.0 0.0 -19 -NULL -13 1969-12-31 16:00:12.905 -13.0 NULL 13 -13 0 -13.0 -0.0 13.0 -1034.1889999999999 0.0 13.0 0.0 -10.175 -13.0 2.0215384615384617 -13.0 0.0 -13 -NULL -31 1969-12-31 16:00:00.755 -31.0 NULL 31 -31 0 -31.0 -0.0 31.0 -2466.143 0.0 31.0 0.0 -10.175 -31.0 0.847741935483871 -31.0 0.0 -31 -true 19 1969-12-31 16:00:12.161 19.0 cvLH6Eat2yFsyy7p -19 19 0 19.0 0.0 -19.0 1511.507 0.0 -19.0 0.0 -10.175 19.0 -1.3831578947368421 19.0 0.0 19 -true 7 1969-12-31 16:00:03.214 7.0 1cGVWH7n1QU -7 7 0 7.0 0.0 -7.0 556.871 0.0 -7.0 0.0 -10.175 7.0 -3.7542857142857144 7.0 0.0 7 -true -11 1969-12-31 16:00:09.351 -11.0 cvLH6Eat2yFsyy7p 11 -11 0 -11.0 -0.0 11.0 -875.083 0.0 11.0 0.0 -10.175 -11.0 2.389090909090909 -11.0 0.0 -11 -NULL 35 1969-12-31 16:00:14.084 35.0 NULL -35 35 0 35.0 0.0 -35.0 2784.355 0.0 -35.0 0.0 -10.175 35.0 -0.7508571428571429 35.0 0.0 35 -NULL -3 1969-12-31 16:00:11.637 -3.0 NULL 3 -3 0 -3.0 -0.0 3.0 -238.659 0.0 3.0 0.0 -10.175 -3.0 8.76 -3.0 0.0 -3 -true -23 1969-12-31 16:00:01.685 -23.0 cvLH6Eat2yFsyy7p 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -true -55 1969-12-31 16:00:13.15 -55.0 1cGVWH7n1QU 55 -55 0 -55.0 -0.0 55.0 -4375.415 0.0 55.0 0.0 -10.175 -55.0 0.47781818181818186 -55.0 0.0 -55 -true -40 1969-12-31 16:00:11.097 -40.0 cvLH6Eat2yFsyy7p 40 -40 0 -40.0 -0.0 40.0 -3182.12 0.0 40.0 0.0 -10.175 -40.0 0.657 -40.0 0.0 -40 -true -44 1969-12-31 16:00:13.489 -44.0 1cGVWH7n1QU 44 -44 0 -44.0 -0.0 44.0 -3500.332 0.0 44.0 0.0 -10.175 -44.0 0.5972727272727273 -44.0 0.0 -44 -true -62 1969-12-31 16:00:07.903 -62.0 821UdmGbkEf4j 62 -62 0 -62.0 -0.0 62.0 -4932.286 0.0 62.0 0.0 -10.175 -62.0 0.4238709677419355 -62.0 0.0 -62 -NULL 58 1969-12-31 16:00:03.239 58.0 NULL -58 58 0 58.0 0.0 -58.0 4614.074 0.0 -58.0 0.0 -10.175 58.0 -0.4531034482758621 58.0 0.0 58 -true 52 1969-12-31 16:00:09.665 52.0 1cGVWH7n1QU -52 52 0 52.0 0.0 -52.0 4136.755999999999 0.0 -52.0 0.0 -10.175 52.0 -0.5053846153846154 52.0 0.0 52 -NULL -12 1969-12-31 16:00:03.541 -12.0 NULL 12 -12 0 -12.0 -0.0 12.0 -954.636 0.0 12.0 0.0 -10.175 -12.0 2.19 -12.0 0.0 -12 -NULL -43 1969-12-31 16:00:15.091 -43.0 NULL 43 -43 0 -43.0 -0.0 43.0 -3420.779 0.0 43.0 0.0 -10.175 -43.0 0.6111627906976744 -43.0 0.0 -43 -true -6 1969-12-31 16:00:14.192 -6.0 1cGVWH7n1QU 6 -6 0 -6.0 -0.0 6.0 -477.318 0.0 6.0 0.0 -10.175 -6.0 4.38 -6.0 0.0 -6 -NULL -49 1969-12-31 16:00:06.851 -49.0 NULL 49 -49 0 -49.0 -0.0 49.0 -3898.0969999999998 0.0 49.0 0.0 -10.175 -49.0 0.5363265306122449 -49.0 0.0 -49 -true -56 1969-12-31 16:00:11.242 -56.0 cvLH6Eat2yFsyy7p 56 -56 0 -56.0 -0.0 56.0 -4454.968 0.0 56.0 0.0 -10.175 -56.0 0.4692857142857143 -56.0 0.0 -56 -NULL -23 1969-12-31 16:00:11.124 -23.0 NULL 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -NULL 19 1969-12-31 16:00:11.771 19.0 NULL -19 19 0 19.0 0.0 -19.0 1511.507 0.0 -19.0 0.0 -10.175 19.0 -1.3831578947368421 19.0 0.0 19 -true 42 1969-12-31 16:00:03.871 42.0 1cGVWH7n1QU -42 42 0 42.0 0.0 -42.0 3341.2259999999997 0.0 -42.0 0.0 -10.175 42.0 -0.6257142857142858 42.0 0.0 42 -true -51 1969-12-31 16:00:14.909 -51.0 1cGVWH7n1QU 51 -51 0 -51.0 -0.0 51.0 -4057.203 0.0 51.0 0.0 -10.175 -51.0 0.5152941176470588 -51.0 0.0 -51 -true -19 1969-12-31 16:00:15.86 -19.0 1cGVWH7n1QU 19 -19 0 -19.0 -0.0 19.0 -1511.507 0.0 19.0 0.0 -10.175 -19.0 1.3831578947368421 -19.0 0.0 -19 -true -63 1969-12-31 16:00:14.899 -63.0 cvLH6Eat2yFsyy7p 63 -63 0 -63.0 -0.0 63.0 -5011.839 0.0 63.0 0.0 -10.175 -63.0 0.41714285714285715 -63.0 0.0 -63 -true 31 1969-12-31 16:00:08.854 31.0 cvLH6Eat2yFsyy7p -31 31 0 31.0 0.0 -31.0 2466.143 0.0 -31.0 0.0 -10.175 31.0 -0.847741935483871 31.0 0.0 31 -true -45 1969-12-31 16:00:00.066 -45.0 1cGVWH7n1QU 45 -45 0 -45.0 -0.0 45.0 -3579.8849999999998 0.0 45.0 0.0 -10.175 -45.0 0.5840000000000001 -45.0 0.0 -45 -true -20 1969-12-31 16:00:09.015 -20.0 1cGVWH7n1QU 20 -20 0 -20.0 -0.0 20.0 -1591.06 0.0 20.0 0.0 -10.175 -20.0 1.314 -20.0 0.0 -20 -true -53 1969-12-31 16:00:09.583 -53.0 821UdmGbkEf4j 53 -53 0 -53.0 -0.0 53.0 -4216.309 0.0 53.0 0.0 -10.175 -53.0 0.4958490566037736 -53.0 0.0 -53 -NULL -54 1969-12-31 16:00:12.181 -54.0 NULL 54 -54 0 -54.0 -0.0 54.0 -4295.862 0.0 54.0 0.0 -10.175 -54.0 0.4866666666666667 -54.0 0.0 -54 -NULL 22 1969-12-31 16:00:09.842 22.0 NULL -22 22 0 22.0 0.0 -22.0 1750.166 0.0 -22.0 0.0 -10.175 22.0 -1.1945454545454546 22.0 0.0 22 -NULL -62 1969-12-31 16:00:14.154 -62.0 NULL 62 -62 0 -62.0 -0.0 62.0 -4932.286 0.0 62.0 0.0 -10.175 -62.0 0.4238709677419355 -62.0 0.0 -62 -NULL -61 1969-12-31 16:00:12.454 -61.0 NULL 61 -61 0 -61.0 -0.0 61.0 -4852.733 0.0 61.0 0.0 -10.175 -61.0 0.4308196721311476 -61.0 0.0 -61 -NULL -9 1969-12-31 16:00:09.792 -9.0 NULL 9 -9 0 -9.0 -0.0 9.0 -715.977 0.0 9.0 0.0 -10.175 -9.0 2.92 -9.0 0.0 -9 -NULL 44 1969-12-31 16:00:03.63 44.0 NULL -44 44 0 44.0 0.0 -44.0 3500.332 0.0 -44.0 0.0 -10.175 44.0 -0.5972727272727273 44.0 0.0 44 -NULL -27 1969-12-31 16:00:08.365 -27.0 NULL 27 -27 0 -27.0 -0.0 27.0 -2147.931 0.0 27.0 0.0 -10.175 -27.0 0.9733333333333334 -27.0 0.0 -27 -NULL -25 1969-12-31 16:00:05.258 -25.0 NULL 25 -25 0 -25.0 -0.0 25.0 -1988.8249999999998 0.0 25.0 0.0 -10.175 -25.0 1.0512000000000001 -25.0 0.0 -25 -true 10 1969-12-31 16:00:08.442 10.0 cvLH6Eat2yFsyy7p -10 10 0 10.0 0.0 -10.0 795.53 0.0 -10.0 0.0 -10.175 10.0 -2.628 10.0 0.0 10 -NULL -24 1969-12-31 16:00:10.682 -24.0 NULL 24 -24 0 -24.0 -0.0 24.0 -1909.272 0.0 24.0 0.0 -10.175 -24.0 1.095 -24.0 0.0 -24 -true 48 1969-12-31 16:00:00.108 48.0 1cGVWH7n1QU -48 48 0 48.0 0.0 -48.0 3818.544 0.0 -48.0 0.0 -10.175 48.0 -0.5475 48.0 0.0 48 -true -34 1969-12-31 16:00:07.5 -34.0 cvLH6Eat2yFsyy7p 34 -34 0 -34.0 -0.0 34.0 -2704.8019999999997 0.0 34.0 0.0 -10.175 -34.0 0.7729411764705882 -34.0 0.0 -34 -true -30 1969-12-31 16:00:04.427 -30.0 1cGVWH7n1QU 30 -30 0 -30.0 -0.0 30.0 -2386.59 0.0 30.0 0.0 -10.175 -30.0 0.876 -30.0 0.0 -30 -true 41 1969-12-31 16:00:06.554 41.0 cvLH6Eat2yFsyy7p -41 41 0 41.0 0.0 -41.0 3261.673 0.0 -41.0 0.0 -10.175 41.0 -0.6409756097560976 41.0 0.0 41 -NULL -2 1969-12-31 16:00:03.174 -2.0 NULL 2 -2 0 -2.0 -0.0 2.0 -159.106 0.0 2.0 0.0 -10.175 -2.0 13.14 -2.0 0.0 -2 -true 46 1969-12-31 16:00:02.31 46.0 cvLH6Eat2yFsyy7p -46 46 0 46.0 0.0 -46.0 3659.438 0.0 -46.0 0.0 -10.175 46.0 -0.571304347826087 46.0 0.0 46 -true -45 1969-12-31 16:00:00.41 -45.0 1cGVWH7n1QU 45 -45 0 -45.0 -0.0 45.0 -3579.8849999999998 0.0 45.0 0.0 -10.175 -45.0 0.5840000000000001 -45.0 0.0 -45 -true 32 1969-12-31 16:00:05.11 32.0 1cGVWH7n1QU -32 32 0 32.0 0.0 -32.0 2545.696 0.0 -32.0 0.0 -10.175 32.0 -0.82125 32.0 0.0 32 -true -63 1969-12-31 16:00:07.623 -63.0 cvLH6Eat2yFsyy7p 63 -63 0 -63.0 -0.0 63.0 -5011.839 0.0 63.0 0.0 -10.175 -63.0 0.41714285714285715 -63.0 0.0 -63 -true 19 1969-12-31 16:00:11.579 19.0 821UdmGbkEf4j -19 19 0 19.0 0.0 -19.0 1511.507 0.0 -19.0 0.0 -10.175 19.0 -1.3831578947368421 19.0 0.0 19 -NULL 60 1969-12-31 16:00:05.691 60.0 NULL -60 60 0 60.0 0.0 -60.0 4773.18 0.0 -60.0 0.0 -10.175 60.0 -0.438 60.0 0.0 60 -true -34 1969-12-31 16:00:14.41 -34.0 821UdmGbkEf4j 34 -34 0 -34.0 -0.0 34.0 -2704.8019999999997 0.0 34.0 0.0 -10.175 -34.0 0.7729411764705882 -34.0 0.0 -34 -true 56 1969-12-31 16:00:00.239 56.0 cvLH6Eat2yFsyy7p -56 56 0 56.0 0.0 -56.0 4454.968 0.0 -56.0 0.0 -10.175 56.0 -0.4692857142857143 56.0 0.0 56 -NULL -41 1969-12-31 16:00:00.951 -41.0 NULL 41 -41 0 -41.0 -0.0 41.0 -3261.673 0.0 41.0 0.0 -10.175 -41.0 0.6409756097560976 -41.0 0.0 -41 -true 29 1969-12-31 16:00:15.007 29.0 cvLH6Eat2yFsyy7p -29 29 0 29.0 0.0 -29.0 2307.037 0.0 -29.0 0.0 -10.175 29.0 -0.9062068965517242 29.0 0.0 29 -true 6 1969-12-31 16:00:04.003 6.0 821UdmGbkEf4j -6 6 0 6.0 0.0 -6.0 477.318 0.0 -6.0 0.0 -10.175 6.0 -4.38 6.0 0.0 6 -NULL 13 1969-12-31 16:00:00.655 13.0 NULL -13 13 0 13.0 0.0 -13.0 1034.1889999999999 0.0 -13.0 0.0 -10.175 13.0 -2.0215384615384617 13.0 0.0 13 -true -48 1969-12-31 16:00:09.472 -48.0 cvLH6Eat2yFsyy7p 48 -48 0 -48.0 -0.0 48.0 -3818.544 0.0 48.0 0.0 -10.175 -48.0 0.5475 -48.0 0.0 -48 -NULL -48 1969-12-31 16:00:13.878 -48.0 NULL 48 -48 0 -48.0 -0.0 48.0 -3818.544 0.0 48.0 0.0 -10.175 -48.0 0.5475 -48.0 0.0 -48 -NULL 2 1969-12-31 16:00:02.196 2.0 NULL -2 2 0 2.0 0.0 -2.0 159.106 0.0 -2.0 0.0 -10.175 2.0 -13.14 2.0 0.0 2 -NULL 47 1969-12-31 16:00:04.997 47.0 NULL -47 47 0 47.0 0.0 -47.0 3738.991 0.0 -47.0 0.0 -10.175 47.0 -0.5591489361702128 47.0 0.0 47 -true 28 1969-12-31 16:00:06.311 28.0 1cGVWH7n1QU -28 28 0 28.0 0.0 -28.0 2227.484 0.0 -28.0 0.0 -10.175 28.0 -0.9385714285714286 28.0 0.0 28 -true 54 1969-12-31 16:00:06.995 54.0 821UdmGbkEf4j -54 54 0 54.0 0.0 -54.0 4295.862 0.0 -54.0 0.0 -10.175 54.0 -0.4866666666666667 54.0 0.0 54 -NULL -59 1969-12-31 16:00:13.625 -59.0 NULL 59 -59 0 -59.0 -0.0 59.0 -4693.6269999999995 0.0 59.0 0.0 -10.175 -59.0 0.44542372881355935 -59.0 0.0 -59 -true 24 1969-12-31 16:00:01.055 24.0 1cGVWH7n1QU -24 24 0 24.0 0.0 -24.0 1909.272 0.0 -24.0 0.0 -10.175 24.0 -1.095 24.0 0.0 24 -true 31 1969-12-31 16:00:12.363 31.0 821UdmGbkEf4j -31 31 0 31.0 0.0 -31.0 2466.143 0.0 -31.0 0.0 -10.175 31.0 -0.847741935483871 31.0 0.0 31 -NULL -10 1969-12-31 16:00:08.882 -10.0 NULL 10 -10 0 -10.0 -0.0 10.0 -795.53 0.0 10.0 0.0 -10.175 -10.0 2.628 -10.0 0.0 -10 -true -56 1969-12-31 16:00:01.467 -56.0 821UdmGbkEf4j 56 -56 0 -56.0 -0.0 56.0 -4454.968 0.0 56.0 0.0 -10.175 -56.0 0.4692857142857143 -56.0 0.0 -56 -true 61 1969-12-31 16:00:05.172 61.0 1cGVWH7n1QU -61 61 0 61.0 0.0 -61.0 4852.733 0.0 -61.0 0.0 -10.175 61.0 -0.4308196721311476 61.0 0.0 61 -true 8 1969-12-31 16:00:07.661 8.0 cvLH6Eat2yFsyy7p -8 8 0 8.0 0.0 -8.0 636.424 0.0 -8.0 0.0 -10.175 8.0 -3.285 8.0 0.0 8 -true 58 1969-12-31 16:00:04.612 58.0 cvLH6Eat2yFsyy7p -58 58 0 58.0 0.0 -58.0 4614.074 0.0 -58.0 0.0 -10.175 58.0 -0.4531034482758621 58.0 0.0 58 -true -19 1969-12-31 16:00:08.34 -19.0 cvLH6Eat2yFsyy7p 19 -19 0 -19.0 -0.0 19.0 -1511.507 0.0 19.0 0.0 -10.175 -19.0 1.3831578947368421 -19.0 0.0 -19 -true -36 1969-12-31 16:00:11.014 -36.0 821UdmGbkEf4j 36 -36 0 -36.0 -0.0 36.0 -2863.908 0.0 36.0 0.0 -10.175 -36.0 0.73 -36.0 0.0 -36 -NULL 9 1969-12-31 16:00:00.684 9.0 NULL -9 9 0 9.0 0.0 -9.0 715.977 0.0 -9.0 0.0 -10.175 9.0 -2.92 9.0 0.0 9 -NULL -34 1969-12-31 16:00:01.396 -34.0 NULL 34 -34 0 -34.0 -0.0 34.0 -2704.8019999999997 0.0 34.0 0.0 -10.175 -34.0 0.7729411764705882 -34.0 0.0 -34 -true -2 1969-12-31 16:00:00.395 -2.0 cvLH6Eat2yFsyy7p 2 -2 0 -2.0 -0.0 2.0 -159.106 0.0 2.0 0.0 -10.175 -2.0 13.14 -2.0 0.0 -2 -NULL -33 1969-12-31 16:00:07.499 -33.0 NULL 33 -33 0 -33.0 -0.0 33.0 -2625.249 0.0 33.0 0.0 -10.175 -33.0 0.7963636363636364 -33.0 0.0 -33 -NULL -35 1969-12-31 16:00:15.507 -35.0 NULL 35 -35 0 -35.0 -0.0 35.0 -2784.355 0.0 35.0 0.0 -10.175 -35.0 0.7508571428571429 -35.0 0.0 -35 -true -51 1969-12-31 16:00:00.231 -51.0 1cGVWH7n1QU 51 -51 0 -51.0 -0.0 51.0 -4057.203 0.0 51.0 0.0 -10.175 -51.0 0.5152941176470588 -51.0 0.0 -51 -NULL -30 1969-12-31 16:00:13.124 -30.0 NULL 30 -30 0 -30.0 -0.0 30.0 -2386.59 0.0 30.0 0.0 -10.175 -30.0 0.876 -30.0 0.0 -30 -NULL 21 1969-12-31 16:00:14.256 21.0 NULL -21 21 0 21.0 0.0 -21.0 1670.6129999999998 0.0 -21.0 0.0 -10.175 21.0 -1.2514285714285716 21.0 0.0 21 -NULL -19 1969-12-31 16:00:04.02 -19.0 NULL 19 -19 0 -19.0 -0.0 19.0 -1511.507 0.0 19.0 0.0 -10.175 -19.0 1.3831578947368421 -19.0 0.0 -19 -true 7 1969-12-31 16:00:15.313 7.0 821UdmGbkEf4j -7 7 0 7.0 0.0 -7.0 556.871 0.0 -7.0 0.0 -10.175 7.0 -3.7542857142857144 7.0 0.0 7 -NULL 34 1969-12-31 16:00:06.34 34.0 NULL -34 34 0 34.0 0.0 -34.0 2704.8019999999997 0.0 -34.0 0.0 -10.175 34.0 -0.7729411764705882 34.0 0.0 34 -true 30 1969-12-31 16:00:04.777 30.0 1cGVWH7n1QU -30 30 0 30.0 0.0 -30.0 2386.59 0.0 -30.0 0.0 -10.175 30.0 -0.876 30.0 0.0 30 -true -37 1969-12-31 16:00:03.236 -37.0 821UdmGbkEf4j 37 -37 0 -37.0 -0.0 37.0 -2943.461 0.0 37.0 0.0 -10.175 -37.0 0.7102702702702703 -37.0 0.0 -37 -true 57 1969-12-31 16:00:11.699 57.0 1cGVWH7n1QU -57 57 0 57.0 0.0 -57.0 4534.521 0.0 -57.0 0.0 -10.175 57.0 -0.4610526315789474 57.0 0.0 57 -NULL 51 1969-12-31 16:00:10.321 51.0 NULL -51 51 0 51.0 0.0 -51.0 4057.203 0.0 -51.0 0.0 -10.175 51.0 -0.5152941176470588 51.0 0.0 51 -true 23 1969-12-31 16:00:06.192 23.0 821UdmGbkEf4j -23 23 0 23.0 0.0 -23.0 1829.719 0.0 -23.0 0.0 -10.175 23.0 -1.142608695652174 23.0 0.0 23 -true -32 1969-12-31 16:00:00.211 -32.0 821UdmGbkEf4j 32 -32 0 -32.0 -0.0 32.0 -2545.696 0.0 32.0 0.0 -10.175 -32.0 0.82125 -32.0 0.0 -32 -NULL 43 1969-12-31 16:00:13.812 43.0 NULL -43 43 0 43.0 0.0 -43.0 3420.779 0.0 -43.0 0.0 -10.175 43.0 -0.6111627906976744 43.0 0.0 43 -true 17 1969-12-31 16:00:02.933 17.0 821UdmGbkEf4j -17 17 0 17.0 0.0 -17.0 1352.4009999999998 0.0 -17.0 0.0 -10.175 17.0 -1.5458823529411765 17.0 0.0 17 -NULL -32 1969-12-31 16:00:08.488 -32.0 NULL 32 -32 0 -32.0 -0.0 32.0 -2545.696 0.0 32.0 0.0 -10.175 -32.0 0.82125 -32.0 0.0 -32 -NULL -4 1969-12-31 16:00:08.129 -4.0 NULL 4 -4 0 -4.0 -0.0 4.0 -318.212 0.0 4.0 0.0 -10.175 -4.0 6.57 -4.0 0.0 -4 -true -50 1969-12-31 16:00:02.251 -50.0 821UdmGbkEf4j 50 -50 0 -50.0 -0.0 50.0 -3977.6499999999996 0.0 50.0 0.0 -10.175 -50.0 0.5256000000000001 -50.0 0.0 -50 -NULL 29 1969-12-31 16:00:05.944 29.0 NULL -29 29 0 29.0 0.0 -29.0 2307.037 0.0 -29.0 0.0 -10.175 29.0 -0.9062068965517242 29.0 0.0 29 -true -22 1969-12-31 16:00:14.444 -22.0 1cGVWH7n1QU 22 -22 0 -22.0 -0.0 22.0 -1750.166 0.0 22.0 0.0 -10.175 -22.0 1.1945454545454546 -22.0 0.0 -22 -NULL -38 1969-12-31 16:00:08.04 -38.0 NULL 38 -38 0 -38.0 -0.0 38.0 -3023.014 0.0 38.0 0.0 -10.175 -38.0 0.6915789473684211 -38.0 0.0 -38 -true -43 1969-12-31 16:00:01.345 -43.0 cvLH6Eat2yFsyy7p 43 -43 0 -43.0 -0.0 43.0 -3420.779 0.0 43.0 0.0 -10.175 -43.0 0.6111627906976744 -43.0 0.0 -43 -true 46 1969-12-31 16:00:09.654 46.0 1cGVWH7n1QU -46 46 0 46.0 0.0 -46.0 3659.438 0.0 -46.0 0.0 -10.175 46.0 -0.571304347826087 46.0 0.0 46 -true 8 1969-12-31 16:00:07.74 8.0 1cGVWH7n1QU -8 8 0 8.0 0.0 -8.0 636.424 0.0 -8.0 0.0 -10.175 8.0 -3.285 8.0 0.0 8 -true -36 1969-12-31 16:00:08.739 -36.0 1cGVWH7n1QU 36 -36 0 -36.0 -0.0 36.0 -2863.908 0.0 36.0 0.0 -10.175 -36.0 0.73 -36.0 0.0 -36 -NULL -60 1969-12-31 16:00:11.996 -60.0 NULL 60 -60 0 -60.0 -0.0 60.0 -4773.18 0.0 60.0 0.0 -10.175 -60.0 0.438 -60.0 0.0 -60 -true -46 1969-12-31 16:00:11.411 -46.0 821UdmGbkEf4j 46 -46 0 -46.0 -0.0 46.0 -3659.438 0.0 46.0 0.0 -10.175 -46.0 0.571304347826087 -46.0 0.0 -46 -NULL -59 1969-12-31 16:00:08.314 -59.0 NULL 59 -59 0 -59.0 -0.0 59.0 -4693.6269999999995 0.0 59.0 0.0 -10.175 -59.0 0.44542372881355935 -59.0 0.0 -59 -true -44 1969-12-31 16:00:00.163 -44.0 cvLH6Eat2yFsyy7p 44 -44 0 -44.0 -0.0 44.0 -3500.332 0.0 44.0 0.0 -10.175 -44.0 0.5972727272727273 -44.0 0.0 -44 -NULL 20 1969-12-31 16:00:11.595 20.0 NULL -20 20 0 20.0 0.0 -20.0 1591.06 0.0 -20.0 0.0 -10.175 20.0 -1.314 20.0 0.0 20 -NULL 2 1969-12-31 16:00:02.752 2.0 NULL -2 2 0 2.0 0.0 -2.0 159.106 0.0 -2.0 0.0 -10.175 2.0 -13.14 2.0 0.0 2 -NULL -64 1969-12-31 16:00:05.997 -64.0 NULL 64 -64 0 -64.0 -0.0 64.0 -5091.392 0.0 64.0 0.0 -10.175 -64.0 0.410625 -64.0 0.0 -64 -true -54 1969-12-31 16:00:03.119 -54.0 cvLH6Eat2yFsyy7p 54 -54 0 -54.0 -0.0 54.0 -4295.862 0.0 54.0 0.0 -10.175 -54.0 0.4866666666666667 -54.0 0.0 -54 -NULL 46 1969-12-31 16:00:00.382 46.0 NULL -46 46 0 46.0 0.0 -46.0 3659.438 0.0 -46.0 0.0 -10.175 46.0 -0.571304347826087 46.0 0.0 46 -true -56 1969-12-31 16:00:06.802 -56.0 821UdmGbkEf4j 56 -56 0 -56.0 -0.0 56.0 -4454.968 0.0 56.0 0.0 -10.175 -56.0 0.4692857142857143 -56.0 0.0 -56 -true -22 1969-12-31 16:00:15.234 -22.0 821UdmGbkEf4j 22 -22 0 -22.0 -0.0 22.0 -1750.166 0.0 22.0 0.0 -10.175 -22.0 1.1945454545454546 -22.0 0.0 -22 -NULL -6 1969-12-31 16:00:14.452 -6.0 NULL 6 -6 0 -6.0 -0.0 6.0 -477.318 0.0 6.0 0.0 -10.175 -6.0 4.38 -6.0 0.0 -6 -true 28 1969-12-31 16:00:00.682 28.0 821UdmGbkEf4j -28 28 0 28.0 0.0 -28.0 2227.484 0.0 -28.0 0.0 -10.175 28.0 -0.9385714285714286 28.0 0.0 28 -true 44 1969-12-31 16:00:09.561 44.0 821UdmGbkEf4j -44 44 0 44.0 0.0 -44.0 3500.332 0.0 -44.0 0.0 -10.175 44.0 -0.5972727272727273 44.0 0.0 44 -NULL -54 1969-12-31 16:00:11.198 -54.0 NULL 54 -54 0 -54.0 -0.0 54.0 -4295.862 0.0 54.0 0.0 -10.175 -54.0 0.4866666666666667 -54.0 0.0 -54 -NULL 57 1969-12-31 16:00:03.351 57.0 NULL -57 57 0 57.0 0.0 -57.0 4534.521 0.0 -57.0 0.0 -10.175 57.0 -0.4610526315789474 57.0 0.0 57 -true 19 1969-12-31 16:00:02.609 19.0 1cGVWH7n1QU -19 19 0 19.0 0.0 -19.0 1511.507 0.0 -19.0 0.0 -10.175 19.0 -1.3831578947368421 19.0 0.0 19 -NULL 59 1969-12-31 16:00:04.476 59.0 NULL -59 59 0 59.0 0.0 -59.0 4693.6269999999995 0.0 -59.0 0.0 -10.175 59.0 -0.44542372881355935 59.0 0.0 59 -true -17 1969-12-31 16:00:07.477 -17.0 cvLH6Eat2yFsyy7p 17 -17 0 -17.0 -0.0 17.0 -1352.4009999999998 0.0 17.0 0.0 -10.175 -17.0 1.5458823529411765 -17.0 0.0 -17 -true -32 1969-12-31 16:00:02.211 -32.0 821UdmGbkEf4j 32 -32 0 -32.0 -0.0 32.0 -2545.696 0.0 32.0 0.0 -10.175 -32.0 0.82125 -32.0 0.0 -32 -NULL -53 1969-12-31 16:00:03.722 -53.0 NULL 53 -53 0 -53.0 -0.0 53.0 -4216.309 0.0 53.0 0.0 -10.175 -53.0 0.4958490566037736 -53.0 0.0 -53 -NULL -32 1969-12-31 16:00:04.577 -32.0 NULL 32 -32 0 -32.0 -0.0 32.0 -2545.696 0.0 32.0 0.0 -10.175 -32.0 0.82125 -32.0 0.0 -32 -NULL 20 1969-12-31 16:00:15.846 20.0 NULL -20 20 0 20.0 0.0 -20.0 1591.06 0.0 -20.0 0.0 -10.175 20.0 -1.314 20.0 0.0 20 -NULL 47 1969-12-31 16:00:06.299 47.0 NULL -47 47 0 47.0 0.0 -47.0 3738.991 0.0 -47.0 0.0 -10.175 47.0 -0.5591489361702128 47.0 0.0 47 -NULL -21 1969-12-31 16:00:01.062 -21.0 NULL 21 -21 0 -21.0 -0.0 21.0 -1670.6129999999998 0.0 21.0 0.0 -10.175 -21.0 1.2514285714285716 -21.0 0.0 -21 -true -53 1969-12-31 16:00:09.276 -53.0 1cGVWH7n1QU 53 -53 0 -53.0 -0.0 53.0 -4216.309 0.0 53.0 0.0 -10.175 -53.0 0.4958490566037736 -53.0 0.0 -53 -true -18 1969-12-31 16:00:09.315 -18.0 cvLH6Eat2yFsyy7p 18 -18 0 -18.0 -0.0 18.0 -1431.954 0.0 18.0 0.0 -10.175 -18.0 1.46 -18.0 0.0 -18 -NULL -40 1969-12-31 16:00:07.83 -40.0 NULL 40 -40 0 -40.0 -0.0 40.0 -3182.12 0.0 40.0 0.0 -10.175 -40.0 0.657 -40.0 0.0 -40 -true -37 1969-12-31 16:00:07.278 -37.0 1cGVWH7n1QU 37 -37 0 -37.0 -0.0 37.0 -2943.461 0.0 37.0 0.0 -10.175 -37.0 0.7102702702702703 -37.0 0.0 -37 -true -7 1969-12-31 16:00:02.17 -7.0 cvLH6Eat2yFsyy7p 7 -7 0 -7.0 -0.0 7.0 -556.871 0.0 7.0 0.0 -10.175 -7.0 3.7542857142857144 -7.0 0.0 -7 -NULL 48 1969-12-31 16:00:00.88 48.0 NULL -48 48 0 48.0 0.0 -48.0 3818.544 0.0 -48.0 0.0 -10.175 48.0 -0.5475 48.0 0.0 48 -true 5 1969-12-31 16:00:10.605 5.0 1cGVWH7n1QU -5 5 0 5.0 0.0 -5.0 397.765 0.0 -5.0 0.0 -10.175 5.0 -5.256 5.0 0.0 5 -NULL -19 1969-12-31 16:00:14.809 -19.0 NULL 19 -19 0 -19.0 -0.0 19.0 -1511.507 0.0 19.0 0.0 -10.175 -19.0 1.3831578947368421 -19.0 0.0 -19 -NULL -30 1969-12-31 16:00:04.236 -30.0 NULL 30 -30 0 -30.0 -0.0 30.0 -2386.59 0.0 30.0 0.0 -10.175 -30.0 0.876 -30.0 0.0 -30 -NULL -3 1969-12-31 16:00:09.819 -3.0 NULL 3 -3 0 -3.0 -0.0 3.0 -238.659 0.0 3.0 0.0 -10.175 -3.0 8.76 -3.0 0.0 -3 -true 25 1969-12-31 16:00:07.493 25.0 821UdmGbkEf4j -25 25 0 25.0 0.0 -25.0 1988.8249999999998 0.0 -25.0 0.0 -10.175 25.0 -1.0512000000000001 25.0 0.0 25 -true -17 1969-12-31 16:00:12.88 -17.0 cvLH6Eat2yFsyy7p 17 -17 0 -17.0 -0.0 17.0 -1352.4009999999998 0.0 17.0 0.0 -10.175 -17.0 1.5458823529411765 -17.0 0.0 -17 -NULL -1 1969-12-31 16:00:00.697 -1.0 NULL 1 -1 0 -1.0 -0.0 1.0 -79.553 0.0 1.0 0.0 -10.175 -1.0 26.28 -1.0 0.0 -1 -true -1 1969-12-31 16:00:16.228 -1.0 821UdmGbkEf4j 1 -1 0 -1.0 -0.0 1.0 -79.553 0.0 1.0 0.0 -10.175 -1.0 26.28 -1.0 0.0 -1 -true 46 1969-12-31 16:00:08.662 46.0 821UdmGbkEf4j -46 46 0 46.0 0.0 -46.0 3659.438 0.0 -46.0 0.0 -10.175 46.0 -0.571304347826087 46.0 0.0 46 -true -44 1969-12-31 16:00:11.092 -44.0 cvLH6Eat2yFsyy7p 44 -44 0 -44.0 -0.0 44.0 -3500.332 0.0 44.0 0.0 -10.175 -44.0 0.5972727272727273 -44.0 0.0 -44 -NULL 54 1969-12-31 16:00:10.45 54.0 NULL -54 54 0 54.0 0.0 -54.0 4295.862 0.0 -54.0 0.0 -10.175 54.0 -0.4866666666666667 54.0 0.0 54 -true 31 1969-12-31 16:00:06.147 31.0 1cGVWH7n1QU -31 31 0 31.0 0.0 -31.0 2466.143 0.0 -31.0 0.0 -10.175 31.0 -0.847741935483871 31.0 0.0 31 -true -64 1969-12-31 16:00:02.496 -64.0 821UdmGbkEf4j 64 -64 0 -64.0 -0.0 64.0 -5091.392 0.0 64.0 0.0 -10.175 -64.0 0.410625 -64.0 0.0 -64 -NULL -7 1969-12-31 16:00:05.452 -7.0 NULL 7 -7 0 -7.0 -0.0 7.0 -556.871 0.0 7.0 0.0 -10.175 -7.0 3.7542857142857144 -7.0 0.0 -7 -true -14 1969-12-31 16:00:03.522 -14.0 821UdmGbkEf4j 14 -14 0 -14.0 -0.0 14.0 -1113.742 0.0 14.0 0.0 -10.175 -14.0 1.8771428571428572 -14.0 0.0 -14 -NULL 36 1969-12-31 16:00:07.27 36.0 NULL -36 36 0 36.0 0.0 -36.0 2863.908 0.0 -36.0 0.0 -10.175 36.0 -0.73 36.0 0.0 36 -NULL 39 1969-12-31 16:00:06.206 39.0 NULL -39 39 0 39.0 0.0 -39.0 3102.567 0.0 -39.0 0.0 -10.175 39.0 -0.6738461538461539 39.0 0.0 39 -true -50 1969-12-31 16:00:01.706 -50.0 cvLH6Eat2yFsyy7p 50 -50 0 -50.0 -0.0 50.0 -3977.6499999999996 0.0 50.0 0.0 -10.175 -50.0 0.5256000000000001 -50.0 0.0 -50 -NULL 26 1969-12-31 16:00:01.871 26.0 NULL -26 26 0 26.0 0.0 -26.0 2068.3779999999997 0.0 -26.0 0.0 -10.175 26.0 -1.0107692307692309 26.0 0.0 26 -true -16 1969-12-31 16:00:00.262 -16.0 821UdmGbkEf4j 16 -16 0 -16.0 -0.0 16.0 -1272.848 0.0 16.0 0.0 -10.175 -16.0 1.6425 -16.0 0.0 -16 -true -53 1969-12-31 16:00:05.412 -53.0 cvLH6Eat2yFsyy7p 53 -53 0 -53.0 -0.0 53.0 -4216.309 0.0 53.0 0.0 -10.175 -53.0 0.4958490566037736 -53.0 0.0 -53 -true 31 1969-12-31 16:00:08.78 31.0 1cGVWH7n1QU -31 31 0 31.0 0.0 -31.0 2466.143 0.0 -31.0 0.0 -10.175 31.0 -0.847741935483871 31.0 0.0 31 -NULL 22 1969-12-31 16:00:06.688 22.0 NULL -22 22 0 22.0 0.0 -22.0 1750.166 0.0 -22.0 0.0 -10.175 22.0 -1.1945454545454546 22.0 0.0 22 -true -62 1969-12-31 16:00:14.872 -62.0 821UdmGbkEf4j 62 -62 0 -62.0 -0.0 62.0 -4932.286 0.0 62.0 0.0 -10.175 -62.0 0.4238709677419355 -62.0 0.0 -62 -NULL -14 1969-12-31 16:00:14.79 -14.0 NULL 14 -14 0 -14.0 -0.0 14.0 -1113.742 0.0 14.0 0.0 -10.175 -14.0 1.8771428571428572 -14.0 0.0 -14 -NULL 35 1969-12-31 16:00:05.007 35.0 NULL -35 35 0 35.0 0.0 -35.0 2784.355 0.0 -35.0 0.0 -10.175 35.0 -0.7508571428571429 35.0 0.0 35 -true 30 1969-12-31 16:00:04.165 30.0 1cGVWH7n1QU -30 30 0 30.0 0.0 -30.0 2386.59 0.0 -30.0 0.0 -10.175 30.0 -0.876 30.0 0.0 30 -NULL -10 1969-12-31 16:00:03.029 -10.0 NULL 10 -10 0 -10.0 -0.0 10.0 -795.53 0.0 10.0 0.0 -10.175 -10.0 2.628 -10.0 0.0 -10 -true -19 1969-12-31 16:00:14.37 -19.0 cvLH6Eat2yFsyy7p 19 -19 0 -19.0 -0.0 19.0 -1511.507 0.0 19.0 0.0 -10.175 -19.0 1.3831578947368421 -19.0 0.0 -19 -true 48 1969-12-31 16:00:06.864 48.0 cvLH6Eat2yFsyy7p -48 48 0 48.0 0.0 -48.0 3818.544 0.0 -48.0 0.0 -10.175 48.0 -0.5475 48.0 0.0 48 -NULL -59 1969-12-31 16:00:10.983 -59.0 NULL 59 -59 0 -59.0 -0.0 59.0 -4693.6269999999995 0.0 59.0 0.0 -10.175 -59.0 0.44542372881355935 -59.0 0.0 -59 -true 49 1969-12-31 16:00:09.365 49.0 cvLH6Eat2yFsyy7p -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 -NULL 4 1969-12-31 16:00:16.08 4.0 NULL -4 4 0 4.0 0.0 -4.0 318.212 0.0 -4.0 0.0 -10.175 4.0 -6.57 4.0 0.0 4 -true 7 1969-12-31 16:00:02.213 7.0 cvLH6Eat2yFsyy7p -7 7 0 7.0 0.0 -7.0 556.871 0.0 -7.0 0.0 -10.175 7.0 -3.7542857142857144 7.0 0.0 7 -NULL 49 1969-12-31 16:00:10.173 49.0 NULL -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 -NULL 1 1969-12-31 16:00:06.965 1.0 NULL -1 1 0 1.0 0.0 -1.0 79.553 0.0 -1.0 0.0 -10.175 1.0 -26.28 1.0 0.0 1 -NULL -19 1969-12-31 16:00:14.83 -19.0 NULL 19 -19 0 -19.0 -0.0 19.0 -1511.507 0.0 19.0 0.0 -10.175 -19.0 1.3831578947368421 -19.0 0.0 -19 -true -38 1969-12-31 16:00:11.661 -38.0 1cGVWH7n1QU 38 -38 0 -38.0 -0.0 38.0 -3023.014 0.0 38.0 0.0 -10.175 -38.0 0.6915789473684211 -38.0 0.0 -38 -NULL -64 1969-12-31 16:00:10.858 -64.0 NULL 64 -64 0 -64.0 -0.0 64.0 -5091.392 0.0 64.0 0.0 -10.175 -64.0 0.410625 -64.0 0.0 -64 -true -3 1969-12-31 16:00:07.182 -3.0 821UdmGbkEf4j 3 -3 0 -3.0 -0.0 3.0 -238.659 0.0 3.0 0.0 -10.175 -3.0 8.76 -3.0 0.0 -3 -true -23 1969-12-31 16:00:13.35 -23.0 cvLH6Eat2yFsyy7p 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -true -48 1969-12-31 16:00:15.716 -48.0 1cGVWH7n1QU 48 -48 0 -48.0 -0.0 48.0 -3818.544 0.0 48.0 0.0 -10.175 -48.0 0.5475 -48.0 0.0 -48 -NULL 14 1969-12-31 16:00:01.952 14.0 NULL -14 14 0 14.0 0.0 -14.0 1113.742 0.0 -14.0 0.0 -10.175 14.0 -1.8771428571428572 14.0 0.0 14 -NULL 14 1969-12-31 16:00:06.3 14.0 NULL -14 14 0 14.0 0.0 -14.0 1113.742 0.0 -14.0 0.0 -10.175 14.0 -1.8771428571428572 14.0 0.0 14 -NULL -46 1969-12-31 16:00:00.601 -46.0 NULL 46 -46 0 -46.0 -0.0 46.0 -3659.438 0.0 46.0 0.0 -10.175 -46.0 0.571304347826087 -46.0 0.0 -46 -true 28 1969-12-31 16:00:09.304 28.0 821UdmGbkEf4j -28 28 0 28.0 0.0 -28.0 2227.484 0.0 -28.0 0.0 -10.175 28.0 -0.9385714285714286 28.0 0.0 28 -NULL -41 1969-12-31 16:00:04.827 -41.0 NULL 41 -41 0 -41.0 -0.0 41.0 -3261.673 0.0 41.0 0.0 -10.175 -41.0 0.6409756097560976 -41.0 0.0 -41 -true 17 1969-12-31 16:00:03.478 17.0 1cGVWH7n1QU -17 17 0 17.0 0.0 -17.0 1352.4009999999998 0.0 -17.0 0.0 -10.175 17.0 -1.5458823529411765 17.0 0.0 17 -true -25 1969-12-31 16:00:00.056 -25.0 1cGVWH7n1QU 25 -25 0 -25.0 -0.0 25.0 -1988.8249999999998 0.0 25.0 0.0 -10.175 -25.0 1.0512000000000001 -25.0 0.0 -25 -NULL -4 1969-12-31 16:00:15.671 -4.0 NULL 4 -4 0 -4.0 -0.0 4.0 -318.212 0.0 4.0 0.0 -10.175 -4.0 6.57 -4.0 0.0 -4 -NULL 16 1969-12-31 16:00:00.649 16.0 NULL -16 16 0 16.0 0.0 -16.0 1272.848 0.0 -16.0 0.0 -10.175 16.0 -1.6425 16.0 0.0 16 -NULL -25 1969-12-31 16:00:06.506 -25.0 NULL 25 -25 0 -25.0 -0.0 25.0 -1988.8249999999998 0.0 25.0 0.0 -10.175 -25.0 1.0512000000000001 -25.0 0.0 -25 -true 9 1969-12-31 16:00:03.354 9.0 1cGVWH7n1QU -9 9 0 9.0 0.0 -9.0 715.977 0.0 -9.0 0.0 -10.175 9.0 -2.92 9.0 0.0 9 -true 32 1969-12-31 16:00:13.909 32.0 cvLH6Eat2yFsyy7p -32 32 0 32.0 0.0 -32.0 2545.696 0.0 -32.0 0.0 -10.175 32.0 -0.82125 32.0 0.0 32 -true 34 1969-12-31 16:00:07.426 34.0 1cGVWH7n1QU -34 34 0 34.0 0.0 -34.0 2704.8019999999997 0.0 -34.0 0.0 -10.175 34.0 -0.7729411764705882 34.0 0.0 34 -NULL -32 1969-12-31 16:00:02.67 -32.0 NULL 32 -32 0 -32.0 -0.0 32.0 -2545.696 0.0 32.0 0.0 -10.175 -32.0 0.82125 -32.0 0.0 -32 -NULL 22 1969-12-31 16:00:00.893 22.0 NULL -22 22 0 22.0 0.0 -22.0 1750.166 0.0 -22.0 0.0 -10.175 22.0 -1.1945454545454546 22.0 0.0 22 -NULL 35 1969-12-31 16:00:00.9 35.0 NULL -35 35 0 35.0 0.0 -35.0 2784.355 0.0 -35.0 0.0 -10.175 35.0 -0.7508571428571429 35.0 0.0 35 -NULL 25 1969-12-31 16:00:07.331 25.0 NULL -25 25 0 25.0 0.0 -25.0 1988.8249999999998 0.0 -25.0 0.0 -10.175 25.0 -1.0512000000000001 25.0 0.0 25 -true -3 1969-12-31 16:00:12.866 -3.0 1cGVWH7n1QU 3 -3 0 -3.0 -0.0 3.0 -238.659 0.0 3.0 0.0 -10.175 -3.0 8.76 -3.0 0.0 -3 -NULL -60 1969-12-31 16:00:03.823 -60.0 NULL 60 -60 0 -60.0 -0.0 60.0 -4773.18 0.0 60.0 0.0 -10.175 -60.0 0.438 -60.0 0.0 -60 -true -53 1969-12-31 16:00:14.582 -53.0 cvLH6Eat2yFsyy7p 53 -53 0 -53.0 -0.0 53.0 -4216.309 0.0 53.0 0.0 -10.175 -53.0 0.4958490566037736 -53.0 0.0 -53 -true 20 1969-12-31 16:00:01.6 20.0 821UdmGbkEf4j -20 20 0 20.0 0.0 -20.0 1591.06 0.0 -20.0 0.0 -10.175 20.0 -1.314 20.0 0.0 20 -true -4 1969-12-31 16:00:05.764 -4.0 821UdmGbkEf4j 4 -4 0 -4.0 -0.0 4.0 -318.212 0.0 4.0 0.0 -10.175 -4.0 6.57 -4.0 0.0 -4 -true 8 1969-12-31 16:00:00.687 8.0 821UdmGbkEf4j -8 8 0 8.0 0.0 -8.0 636.424 0.0 -8.0 0.0 -10.175 8.0 -3.285 8.0 0.0 8 -NULL 38 1969-12-31 16:00:08.703 38.0 NULL -38 38 0 38.0 0.0 -38.0 3023.014 0.0 -38.0 0.0 -10.175 38.0 -0.6915789473684211 38.0 0.0 38 -true -2 1969-12-31 16:00:03.27 -2.0 1cGVWH7n1QU 2 -2 0 -2.0 -0.0 2.0 -159.106 0.0 2.0 0.0 -10.175 -2.0 13.14 -2.0 0.0 -2 -NULL 37 1969-12-31 16:00:12.712 37.0 NULL -37 37 0 37.0 0.0 -37.0 2943.461 0.0 -37.0 0.0 -10.175 37.0 -0.7102702702702703 37.0 0.0 37 -true 7 1969-12-31 16:00:03.846 7.0 cvLH6Eat2yFsyy7p -7 7 0 7.0 0.0 -7.0 556.871 0.0 -7.0 0.0 -10.175 7.0 -3.7542857142857144 7.0 0.0 7 -NULL 41 1969-12-31 16:00:07.175 41.0 NULL -41 41 0 41.0 0.0 -41.0 3261.673 0.0 -41.0 0.0 -10.175 41.0 -0.6409756097560976 41.0 0.0 41 -NULL -55 1969-12-31 16:00:05.195 -55.0 NULL 55 -55 0 -55.0 -0.0 55.0 -4375.415 0.0 55.0 0.0 -10.175 -55.0 0.47781818181818186 -55.0 0.0 -55 -true -9 1969-12-31 16:00:10.558 -9.0 cvLH6Eat2yFsyy7p 9 -9 0 -9.0 -0.0 9.0 -715.977 0.0 9.0 0.0 -10.175 -9.0 2.92 -9.0 0.0 -9 -true 35 1969-12-31 16:00:10.916 35.0 1cGVWH7n1QU -35 35 0 35.0 0.0 -35.0 2784.355 0.0 -35.0 0.0 -10.175 35.0 -0.7508571428571429 35.0 0.0 35 -true 6 1969-12-31 16:00:03.916 6.0 1cGVWH7n1QU -6 6 0 6.0 0.0 -6.0 477.318 0.0 -6.0 0.0 -10.175 6.0 -4.38 6.0 0.0 6 -NULL 40 1969-12-31 16:00:14.565 40.0 NULL -40 40 0 40.0 0.0 -40.0 3182.12 0.0 -40.0 0.0 -10.175 40.0 -0.657 40.0 0.0 40 -NULL -19 1969-12-31 16:00:02.925 -19.0 NULL 19 -19 0 -19.0 -0.0 19.0 -1511.507 0.0 19.0 0.0 -10.175 -19.0 1.3831578947368421 -19.0 0.0 -19 -true -14 1969-12-31 16:00:08.605 -14.0 1cGVWH7n1QU 14 -14 0 -14.0 -0.0 14.0 -1113.742 0.0 14.0 0.0 -10.175 -14.0 1.8771428571428572 -14.0 0.0 -14 -true 37 1969-12-31 16:00:04.828 37.0 821UdmGbkEf4j -37 37 0 37.0 0.0 -37.0 2943.461 0.0 -37.0 0.0 -10.175 37.0 -0.7102702702702703 37.0 0.0 37 -true -1 1969-12-31 16:00:09.723 -1.0 cvLH6Eat2yFsyy7p 1 -1 0 -1.0 -0.0 1.0 -79.553 0.0 1.0 0.0 -10.175 -1.0 26.28 -1.0 0.0 -1 -true 30 1969-12-31 16:00:06.183 30.0 821UdmGbkEf4j -30 30 0 30.0 0.0 -30.0 2386.59 0.0 -30.0 0.0 -10.175 30.0 -0.876 30.0 0.0 30 -NULL -24 1969-12-31 16:00:10.383 -24.0 NULL 24 -24 0 -24.0 -0.0 24.0 -1909.272 0.0 24.0 0.0 -10.175 -24.0 1.095 -24.0 0.0 -24 -true 34 1969-12-31 16:00:07.279 34.0 821UdmGbkEf4j -34 34 0 34.0 0.0 -34.0 2704.8019999999997 0.0 -34.0 0.0 -10.175 34.0 -0.7729411764705882 34.0 0.0 34 -NULL -64 1969-12-31 16:00:12.339 -64.0 NULL 64 -64 0 -64.0 -0.0 64.0 -5091.392 0.0 64.0 0.0 -10.175 -64.0 0.410625 -64.0 0.0 -64 -true -6 1969-12-31 16:00:08.798 -6.0 1cGVWH7n1QU 6 -6 0 -6.0 -0.0 6.0 -477.318 0.0 6.0 0.0 -10.175 -6.0 4.38 -6.0 0.0 -6 -NULL -16 1969-12-31 16:00:06.481 -16.0 NULL 16 -16 0 -16.0 -0.0 16.0 -1272.848 0.0 16.0 0.0 -10.175 -16.0 1.6425 -16.0 0.0 -16 -NULL 42 1969-12-31 16:00:16.037 42.0 NULL -42 42 0 42.0 0.0 -42.0 3341.2259999999997 0.0 -42.0 0.0 -10.175 42.0 -0.6257142857142858 42.0 0.0 42 -true 17 1969-12-31 16:00:04.153 17.0 821UdmGbkEf4j -17 17 0 17.0 0.0 -17.0 1352.4009999999998 0.0 -17.0 0.0 -10.175 17.0 -1.5458823529411765 17.0 0.0 17 -true 19 1969-12-31 16:00:13.106 19.0 1cGVWH7n1QU -19 19 0 19.0 0.0 -19.0 1511.507 0.0 -19.0 0.0 -10.175 19.0 -1.3831578947368421 19.0 0.0 19 -NULL -42 1969-12-31 16:00:12.041 -42.0 NULL 42 -42 0 -42.0 -0.0 42.0 -3341.2259999999997 0.0 42.0 0.0 -10.175 -42.0 0.6257142857142858 -42.0 0.0 -42 -true -9 1969-12-31 16:00:10.035 -9.0 1cGVWH7n1QU 9 -9 0 -9.0 -0.0 9.0 -715.977 0.0 9.0 0.0 -10.175 -9.0 2.92 -9.0 0.0 -9 -true -57 1969-12-31 16:00:09.066 -57.0 1cGVWH7n1QU 57 -57 0 -57.0 -0.0 57.0 -4534.521 0.0 57.0 0.0 -10.175 -57.0 0.4610526315789474 -57.0 0.0 -57 -NULL -12 1969-12-31 16:00:02.93 -12.0 NULL 12 -12 0 -12.0 -0.0 12.0 -954.636 0.0 12.0 0.0 -10.175 -12.0 2.19 -12.0 0.0 -12 -NULL -30 1969-12-31 16:00:08.346 -30.0 NULL 30 -30 0 -30.0 -0.0 30.0 -2386.59 0.0 30.0 0.0 -10.175 -30.0 0.876 -30.0 0.0 -30 -true -36 1969-12-31 16:00:07.214 -36.0 1cGVWH7n1QU 36 -36 0 -36.0 -0.0 36.0 -2863.908 0.0 36.0 0.0 -10.175 -36.0 0.73 -36.0 0.0 -36 -NULL 37 1969-12-31 16:00:11.091 37.0 NULL -37 37 0 37.0 0.0 -37.0 2943.461 0.0 -37.0 0.0 -10.175 37.0 -0.7102702702702703 37.0 0.0 37 -true 8 1969-12-31 16:00:11.236 8.0 1cGVWH7n1QU -8 8 0 8.0 0.0 -8.0 636.424 0.0 -8.0 0.0 -10.175 8.0 -3.285 8.0 0.0 8 -NULL 0 1969-12-31 16:00:00.437 0.0 NULL 0 0 0 0.0 0.0 -0.0 0.0 0.0 -0.0 0.0 -10.175 0.0 NULL 0.0 NULL 0 -NULL 41 1969-12-31 16:00:09.951 41.0 NULL -41 41 0 41.0 0.0 -41.0 3261.673 0.0 -41.0 0.0 -10.175 41.0 -0.6409756097560976 41.0 0.0 41 -NULL 24 1969-12-31 16:00:12.66 24.0 NULL -24 24 0 24.0 0.0 -24.0 1909.272 0.0 -24.0 0.0 -10.175 24.0 -1.095 24.0 0.0 24 -true -38 1969-12-31 16:00:10.952 -38.0 cvLH6Eat2yFsyy7p 38 -38 0 -38.0 -0.0 38.0 -3023.014 0.0 38.0 0.0 -10.175 -38.0 0.6915789473684211 -38.0 0.0 -38 -NULL -36 1969-12-31 16:00:15.62 -36.0 NULL 36 -36 0 -36.0 -0.0 36.0 -2863.908 0.0 36.0 0.0 -10.175 -36.0 0.73 -36.0 0.0 -36 -true -34 1969-12-31 16:00:11.334 -34.0 cvLH6Eat2yFsyy7p 34 -34 0 -34.0 -0.0 34.0 -2704.8019999999997 0.0 34.0 0.0 -10.175 -34.0 0.7729411764705882 -34.0 0.0 -34 -NULL 45 1969-12-31 16:00:00.663 45.0 NULL -45 45 0 45.0 0.0 -45.0 3579.8849999999998 0.0 -45.0 0.0 -10.175 45.0 -0.5840000000000001 45.0 0.0 45 -true 42 1969-12-31 16:00:12.024 42.0 821UdmGbkEf4j -42 42 0 42.0 0.0 -42.0 3341.2259999999997 0.0 -42.0 0.0 -10.175 42.0 -0.6257142857142858 42.0 0.0 42 -true 0 1969-12-31 16:00:08.742 0.0 821UdmGbkEf4j 0 0 0 0.0 0.0 -0.0 0.0 0.0 -0.0 0.0 -10.175 0.0 NULL 0.0 NULL 0 -NULL -40 1969-12-31 16:00:07.622 -40.0 NULL 40 -40 0 -40.0 -0.0 40.0 -3182.12 0.0 40.0 0.0 -10.175 -40.0 0.657 -40.0 0.0 -40 -NULL -22 1969-12-31 16:00:15.409 -22.0 NULL 22 -22 0 -22.0 -0.0 22.0 -1750.166 0.0 22.0 0.0 -10.175 -22.0 1.1945454545454546 -22.0 0.0 -22 -NULL 10 1969-12-31 16:00:15.298 10.0 NULL -10 10 0 10.0 0.0 -10.0 795.53 0.0 -10.0 0.0 -10.175 10.0 -2.628 10.0 0.0 10 -true -31 1969-12-31 16:00:10.861 -31.0 821UdmGbkEf4j 31 -31 0 -31.0 -0.0 31.0 -2466.143 0.0 31.0 0.0 -10.175 -31.0 0.847741935483871 -31.0 0.0 -31 -true 41 1969-12-31 16:00:04.306 41.0 821UdmGbkEf4j -41 41 0 41.0 0.0 -41.0 3261.673 0.0 -41.0 0.0 -10.175 41.0 -0.6409756097560976 41.0 0.0 41 -true 7 1969-12-31 16:00:08.334 7.0 1cGVWH7n1QU -7 7 0 7.0 0.0 -7.0 556.871 0.0 -7.0 0.0 -10.175 7.0 -3.7542857142857144 7.0 0.0 7 -NULL 14 1969-12-31 16:00:00.394 14.0 NULL -14 14 0 14.0 0.0 -14.0 1113.742 0.0 -14.0 0.0 -10.175 14.0 -1.8771428571428572 14.0 0.0 14 -true 23 1969-12-31 16:00:06.967 23.0 cvLH6Eat2yFsyy7p -23 23 0 23.0 0.0 -23.0 1829.719 0.0 -23.0 0.0 -10.175 23.0 -1.142608695652174 23.0 0.0 23 -NULL -30 1969-12-31 16:00:16.178 -30.0 NULL 30 -30 0 -30.0 -0.0 30.0 -2386.59 0.0 30.0 0.0 -10.175 -30.0 0.876 -30.0 0.0 -30 -NULL 58 1969-12-31 16:00:04.21 58.0 NULL -58 58 0 58.0 0.0 -58.0 4614.074 0.0 -58.0 0.0 -10.175 58.0 -0.4531034482758621 58.0 0.0 58 -NULL -44 1969-12-31 16:00:00.247 -44.0 NULL 44 -44 0 -44.0 -0.0 44.0 -3500.332 0.0 44.0 0.0 -10.175 -44.0 0.5972727272727273 -44.0 0.0 -44 -NULL 49 1969-12-31 16:00:13.879 49.0 NULL -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 -true -32 1969-12-31 16:00:09.512 -32.0 cvLH6Eat2yFsyy7p 32 -32 0 -32.0 -0.0 32.0 -2545.696 0.0 32.0 0.0 -10.175 -32.0 0.82125 -32.0 0.0 -32 -NULL 59 1969-12-31 16:00:15.05 59.0 NULL -59 59 0 59.0 0.0 -59.0 4693.6269999999995 0.0 -59.0 0.0 -10.175 59.0 -0.44542372881355935 59.0 0.0 59 -NULL -44 1969-12-31 16:00:14.237 -44.0 NULL 44 -44 0 -44.0 -0.0 44.0 -3500.332 0.0 44.0 0.0 -10.175 -44.0 0.5972727272727273 -44.0 0.0 -44 -true 47 1969-12-31 16:00:00.803 47.0 cvLH6Eat2yFsyy7p -47 47 0 47.0 0.0 -47.0 3738.991 0.0 -47.0 0.0 -10.175 47.0 -0.5591489361702128 47.0 0.0 47 -true -15 1969-12-31 16:00:13.271 -15.0 821UdmGbkEf4j 15 -15 0 -15.0 -0.0 15.0 -1193.295 0.0 15.0 0.0 -10.175 -15.0 1.752 -15.0 0.0 -15 -true 13 1969-12-31 16:00:13.062 13.0 cvLH6Eat2yFsyy7p -13 13 0 13.0 0.0 -13.0 1034.1889999999999 0.0 -13.0 0.0 -10.175 13.0 -2.0215384615384617 13.0 0.0 13 -true -7 1969-12-31 16:00:01.603 -7.0 821UdmGbkEf4j 7 -7 0 -7.0 -0.0 7.0 -556.871 0.0 7.0 0.0 -10.175 -7.0 3.7542857142857144 -7.0 0.0 -7 -true 18 1969-12-31 16:00:00.909 18.0 cvLH6Eat2yFsyy7p -18 18 0 18.0 0.0 -18.0 1431.954 0.0 -18.0 0.0 -10.175 18.0 -1.46 18.0 0.0 18 -true 25 1969-12-31 16:00:00.094 25.0 cvLH6Eat2yFsyy7p -25 25 0 25.0 0.0 -25.0 1988.8249999999998 0.0 -25.0 0.0 -10.175 25.0 -1.0512000000000001 25.0 0.0 25 -NULL 34 1969-12-31 16:00:08.234 34.0 NULL -34 34 0 34.0 0.0 -34.0 2704.8019999999997 0.0 -34.0 0.0 -10.175 34.0 -0.7729411764705882 34.0 0.0 34 -NULL -27 1969-12-31 16:00:02.92 -27.0 NULL 27 -27 0 -27.0 -0.0 27.0 -2147.931 0.0 27.0 0.0 -10.175 -27.0 0.9733333333333334 -27.0 0.0 -27 -NULL 4 1969-12-31 16:00:00.019 4.0 NULL -4 4 0 4.0 0.0 -4.0 318.212 0.0 -4.0 0.0 -10.175 4.0 -6.57 4.0 0.0 4 -NULL -43 1969-12-31 16:00:02.393 -43.0 NULL 43 -43 0 -43.0 -0.0 43.0 -3420.779 0.0 43.0 0.0 -10.175 -43.0 0.6111627906976744 -43.0 0.0 -43 -true -38 1969-12-31 16:00:00.1 -38.0 1cGVWH7n1QU 38 -38 0 -38.0 -0.0 38.0 -3023.014 0.0 38.0 0.0 -10.175 -38.0 0.6915789473684211 -38.0 0.0 -38 -NULL -29 1969-12-31 16:00:16.065 -29.0 NULL 29 -29 0 -29.0 -0.0 29.0 -2307.037 0.0 29.0 0.0 -10.175 -29.0 0.9062068965517242 -29.0 0.0 -29 -true -33 1969-12-31 16:00:03.168 -33.0 cvLH6Eat2yFsyy7p 33 -33 0 -33.0 -0.0 33.0 -2625.249 0.0 33.0 0.0 -10.175 -33.0 0.7963636363636364 -33.0 0.0 -33 -NULL 39 1969-12-31 16:00:07.541 39.0 NULL -39 39 0 39.0 0.0 -39.0 3102.567 0.0 -39.0 0.0 -10.175 39.0 -0.6738461538461539 39.0 0.0 39 -NULL -37 1969-12-31 16:00:00.539 -37.0 NULL 37 -37 0 -37.0 -0.0 37.0 -2943.461 0.0 37.0 0.0 -10.175 -37.0 0.7102702702702703 -37.0 0.0 -37 -NULL 59 1969-12-31 16:00:00.738 59.0 NULL -59 59 0 59.0 0.0 -59.0 4693.6269999999995 0.0 -59.0 0.0 -10.175 59.0 -0.44542372881355935 59.0 0.0 59 -true -32 1969-12-31 16:00:09.404 -32.0 cvLH6Eat2yFsyy7p 32 -32 0 -32.0 -0.0 32.0 -2545.696 0.0 32.0 0.0 -10.175 -32.0 0.82125 -32.0 0.0 -32 -NULL -61 1969-12-31 16:00:10.239 -61.0 NULL 61 -61 0 -61.0 -0.0 61.0 -4852.733 0.0 61.0 0.0 -10.175 -61.0 0.4308196721311476 -61.0 0.0 -61 -true 62 1969-12-31 16:00:09.455 62.0 821UdmGbkEf4j -62 62 0 62.0 0.0 -62.0 4932.286 0.0 -62.0 0.0 -10.175 62.0 -0.4238709677419355 62.0 0.0 62 -true 35 1969-12-31 16:00:14.07 35.0 1cGVWH7n1QU -35 35 0 35.0 0.0 -35.0 2784.355 0.0 -35.0 0.0 -10.175 35.0 -0.7508571428571429 35.0 0.0 35 -true -17 1969-12-31 16:00:02.925 -17.0 cvLH6Eat2yFsyy7p 17 -17 0 -17.0 -0.0 17.0 -1352.4009999999998 0.0 17.0 0.0 -10.175 -17.0 1.5458823529411765 -17.0 0.0 -17 -true -47 1969-12-31 16:00:03.722 -47.0 821UdmGbkEf4j 47 -47 0 -47.0 -0.0 47.0 -3738.991 0.0 47.0 0.0 -10.175 -47.0 0.5591489361702128 -47.0 0.0 -47 -NULL 46 1969-12-31 16:00:07.743 46.0 NULL -46 46 0 46.0 0.0 -46.0 3659.438 0.0 -46.0 0.0 -10.175 46.0 -0.571304347826087 46.0 0.0 46 -NULL -55 1969-12-31 16:00:00.476 -55.0 NULL 55 -55 0 -55.0 -0.0 55.0 -4375.415 0.0 55.0 0.0 -10.175 -55.0 0.47781818181818186 -55.0 0.0 -55 -true 42 1969-12-31 16:00:03.797 42.0 821UdmGbkEf4j -42 42 0 42.0 0.0 -42.0 3341.2259999999997 0.0 -42.0 0.0 -10.175 42.0 -0.6257142857142858 42.0 0.0 42 -true -7 1969-12-31 16:00:01.162 -7.0 1cGVWH7n1QU 7 -7 0 -7.0 -0.0 7.0 -556.871 0.0 7.0 0.0 -10.175 -7.0 3.7542857142857144 -7.0 0.0 -7 -true -64 1969-12-31 16:00:12.857 -64.0 821UdmGbkEf4j 64 -64 0 -64.0 -0.0 64.0 -5091.392 0.0 64.0 0.0 -10.175 -64.0 0.410625 -64.0 0.0 -64 -NULL 25 1969-12-31 16:00:00.73 25.0 NULL -25 25 0 25.0 0.0 -25.0 1988.8249999999998 0.0 -25.0 0.0 -10.175 25.0 -1.0512000000000001 25.0 0.0 25 -true -43 1969-12-31 16:00:05.983 -43.0 1cGVWH7n1QU 43 -43 0 -43.0 -0.0 43.0 -3420.779 0.0 43.0 0.0 -10.175 -43.0 0.6111627906976744 -43.0 0.0 -43 -NULL 34 1969-12-31 16:00:15.609 34.0 NULL -34 34 0 34.0 0.0 -34.0 2704.8019999999997 0.0 -34.0 0.0 -10.175 34.0 -0.7729411764705882 34.0 0.0 34 -NULL 4 1969-12-31 16:00:00.563 4.0 NULL -4 4 0 4.0 0.0 -4.0 318.212 0.0 -4.0 0.0 -10.175 4.0 -6.57 4.0 0.0 4 -true -2 1969-12-31 16:00:09.237 -2.0 cvLH6Eat2yFsyy7p 2 -2 0 -2.0 -0.0 2.0 -159.106 0.0 2.0 0.0 -10.175 -2.0 13.14 -2.0 0.0 -2 -true 37 1969-12-31 16:00:00.829 37.0 cvLH6Eat2yFsyy7p -37 37 0 37.0 0.0 -37.0 2943.461 0.0 -37.0 0.0 -10.175 37.0 -0.7102702702702703 37.0 0.0 37 -true -24 1969-12-31 16:00:12.17 -24.0 cvLH6Eat2yFsyy7p 24 -24 0 -24.0 -0.0 24.0 -1909.272 0.0 24.0 0.0 -10.175 -24.0 1.095 -24.0 0.0 -24 -NULL 33 1969-12-31 16:00:00.761 79.553 NULL -33 33 0 79.5530014038086 0.0 -79.5530014038086 6328.679920677185 0.0 -79.5530014038086 0.0 -10.175 79.5530014038086 -0.33034580136836733 79.553 0.0 33 -true 55 1969-12-31 16:00:08.465 55.0 cvLH6Eat2yFsyy7p -55 55 0 55.0 0.0 -55.0 4375.415 0.0 -55.0 0.0 -10.175 55.0 -0.47781818181818186 55.0 0.0 55 -true -24 1969-12-31 16:00:03.22 -24.0 821UdmGbkEf4j 24 -24 0 -24.0 -0.0 24.0 -1909.272 0.0 24.0 0.0 -10.175 -24.0 1.095 -24.0 0.0 -24 -true 39 1969-12-31 16:00:07.162 39.0 cvLH6Eat2yFsyy7p -39 39 0 39.0 0.0 -39.0 3102.567 0.0 -39.0 0.0 -10.175 39.0 -0.6738461538461539 39.0 0.0 39 -NULL 35 1969-12-31 16:00:08.58 35.0 NULL -35 35 0 35.0 0.0 -35.0 2784.355 0.0 -35.0 0.0 -10.175 35.0 -0.7508571428571429 35.0 0.0 35 -true 57 1969-12-31 16:00:11.286 57.0 821UdmGbkEf4j -57 57 0 57.0 0.0 -57.0 4534.521 0.0 -57.0 0.0 -10.175 57.0 -0.4610526315789474 57.0 0.0 57 -true -54 1969-12-31 16:00:03.725 -54.0 1cGVWH7n1QU 54 -54 0 -54.0 -0.0 54.0 -4295.862 0.0 54.0 0.0 -10.175 -54.0 0.4866666666666667 -54.0 0.0 -54 -NULL -6 1969-12-31 16:00:11.883 -6.0 NULL 6 -6 0 -6.0 -0.0 6.0 -477.318 0.0 6.0 0.0 -10.175 -6.0 4.38 -6.0 0.0 -6 -NULL 49 1969-12-31 16:00:00.805 49.0 NULL -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 -NULL 5 1969-12-31 16:00:13.574 5.0 NULL -5 5 0 5.0 0.0 -5.0 397.765 0.0 -5.0 0.0 -10.175 5.0 -5.256 5.0 0.0 5 -NULL 5 1969-12-31 16:00:00.49 5.0 NULL -5 5 0 5.0 0.0 -5.0 397.765 0.0 -5.0 0.0 -10.175 5.0 -5.256 5.0 0.0 5 -true 40 1969-12-31 16:00:08.785 40.0 1cGVWH7n1QU -40 40 0 40.0 0.0 -40.0 3182.12 0.0 -40.0 0.0 -10.175 40.0 -0.657 40.0 0.0 40 -true 23 1969-12-31 16:00:15.551 23.0 cvLH6Eat2yFsyy7p -23 23 0 23.0 0.0 -23.0 1829.719 0.0 -23.0 0.0 -10.175 23.0 -1.142608695652174 23.0 0.0 23 -NULL -58 1969-12-31 16:00:02.972 -58.0 NULL 58 -58 0 -58.0 -0.0 58.0 -4614.074 0.0 58.0 0.0 -10.175 -58.0 0.4531034482758621 -58.0 0.0 -58 -NULL -16 1969-12-31 16:00:03.189 -16.0 NULL 16 -16 0 -16.0 -0.0 16.0 -1272.848 0.0 16.0 0.0 -10.175 -16.0 1.6425 -16.0 0.0 -16 -true 13 1969-12-31 16:00:02.001 13.0 1cGVWH7n1QU -13 13 0 13.0 0.0 -13.0 1034.1889999999999 0.0 -13.0 0.0 -10.175 13.0 -2.0215384615384617 13.0 0.0 13 -true 59 1969-12-31 16:00:00.373 59.0 cvLH6Eat2yFsyy7p -59 59 0 59.0 0.0 -59.0 4693.6269999999995 0.0 -59.0 0.0 -10.175 59.0 -0.44542372881355935 59.0 0.0 59 -NULL -57 1969-12-31 16:00:13.578 -57.0 NULL 57 -57 0 -57.0 -0.0 57.0 -4534.521 0.0 57.0 0.0 -10.175 -57.0 0.4610526315789474 -57.0 0.0 -57 -NULL -50 1969-12-31 16:00:05.725 -50.0 NULL 50 -50 0 -50.0 -0.0 50.0 -3977.6499999999996 0.0 50.0 0.0 -10.175 -50.0 0.5256000000000001 -50.0 0.0 -50 -NULL 12 1969-12-31 16:00:00.674 12.0 NULL -12 12 0 12.0 0.0 -12.0 954.636 0.0 -12.0 0.0 -10.175 12.0 -2.19 12.0 0.0 12 -true -43 1969-12-31 16:00:09.786 -43.0 821UdmGbkEf4j 43 -43 0 -43.0 -0.0 43.0 -3420.779 0.0 43.0 0.0 -10.175 -43.0 0.6111627906976744 -43.0 0.0 -43 -true -18 1969-12-31 16:00:12.593 -18.0 cvLH6Eat2yFsyy7p 18 -18 0 -18.0 -0.0 18.0 -1431.954 0.0 18.0 0.0 -10.175 -18.0 1.46 -18.0 0.0 -18 -true 2 1969-12-31 16:00:07.556 2.0 cvLH6Eat2yFsyy7p -2 2 0 2.0 0.0 -2.0 159.106 0.0 -2.0 0.0 -10.175 2.0 -13.14 2.0 0.0 2 -true -11 1969-12-31 16:00:12.484 -11.0 1cGVWH7n1QU 11 -11 0 -11.0 -0.0 11.0 -875.083 0.0 11.0 0.0 -10.175 -11.0 2.389090909090909 -11.0 0.0 -11 -true 50 1969-12-31 16:00:11.087 50.0 1cGVWH7n1QU -50 50 0 50.0 0.0 -50.0 3977.6499999999996 0.0 -50.0 0.0 -10.175 50.0 -0.5256000000000001 50.0 0.0 50 -NULL 5 1969-12-31 16:00:11.642 5.0 NULL -5 5 0 5.0 0.0 -5.0 397.765 0.0 -5.0 0.0 -10.175 5.0 -5.256 5.0 0.0 5 -NULL -23 1969-12-31 16:00:01.813 -23.0 NULL 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -true -23 1969-12-31 16:00:03.774 -23.0 1cGVWH7n1QU 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -NULL 59 1969-12-31 16:00:10.956 59.0 NULL -59 59 0 59.0 0.0 -59.0 4693.6269999999995 0.0 -59.0 0.0 -10.175 59.0 -0.44542372881355935 59.0 0.0 59 -true -5 1969-12-31 16:00:13.518 -5.0 1cGVWH7n1QU 5 -5 0 -5.0 -0.0 5.0 -397.765 0.0 5.0 0.0 -10.175 -5.0 5.256 -5.0 0.0 -5 -NULL 29 1969-12-31 16:00:13.455 29.0 NULL -29 29 0 29.0 0.0 -29.0 2307.037 0.0 -29.0 0.0 -10.175 29.0 -0.9062068965517242 29.0 0.0 29 -true -40 1969-12-31 16:00:03.604 -40.0 1cGVWH7n1QU 40 -40 0 -40.0 -0.0 40.0 -3182.12 0.0 40.0 0.0 -10.175 -40.0 0.657 -40.0 0.0 -40 -true 36 1969-12-31 16:00:02.752 36.0 1cGVWH7n1QU -36 36 0 36.0 0.0 -36.0 2863.908 0.0 -36.0 0.0 -10.175 36.0 -0.73 36.0 0.0 36 -true 17 1969-12-31 16:00:14.12 17.0 cvLH6Eat2yFsyy7p -17 17 0 17.0 0.0 -17.0 1352.4009999999998 0.0 -17.0 0.0 -10.175 17.0 -1.5458823529411765 17.0 0.0 17 -NULL 56 1969-12-31 16:00:16.056 56.0 NULL -56 56 0 56.0 0.0 -56.0 4454.968 0.0 -56.0 0.0 -10.175 56.0 -0.4692857142857143 56.0 0.0 56 -true -41 1969-12-31 16:00:15.219 -41.0 1cGVWH7n1QU 41 -41 0 -41.0 -0.0 41.0 -3261.673 0.0 41.0 0.0 -10.175 -41.0 0.6409756097560976 -41.0 0.0 -41 -NULL 54 1969-12-31 16:00:10.187 54.0 NULL -54 54 0 54.0 0.0 -54.0 4295.862 0.0 -54.0 0.0 -10.175 54.0 -0.4866666666666667 54.0 0.0 54 -true -23 1969-12-31 16:00:06.775 -23.0 821UdmGbkEf4j 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -NULL 22 1969-12-31 16:00:05.334 22.0 NULL -22 22 0 22.0 0.0 -22.0 1750.166 0.0 -22.0 0.0 -10.175 22.0 -1.1945454545454546 22.0 0.0 22 -NULL -22 1969-12-31 16:00:10.132 -22.0 NULL 22 -22 0 -22.0 -0.0 22.0 -1750.166 0.0 22.0 0.0 -10.175 -22.0 1.1945454545454546 -22.0 0.0 -22 -true 12 1969-12-31 16:00:14.744 12.0 821UdmGbkEf4j -12 12 0 12.0 0.0 -12.0 954.636 0.0 -12.0 0.0 -10.175 12.0 -2.19 12.0 0.0 12 -NULL -7 1969-12-31 16:00:13.029 -7.0 NULL 7 -7 0 -7.0 -0.0 7.0 -556.871 0.0 7.0 0.0 -10.175 -7.0 3.7542857142857144 -7.0 0.0 -7 -NULL -3 1969-12-31 16:00:16.338 -3.0 NULL 3 -3 0 -3.0 -0.0 3.0 -238.659 0.0 3.0 0.0 -10.175 -3.0 8.76 -3.0 0.0 -3 -true 14 1969-12-31 16:00:08.209 14.0 1cGVWH7n1QU -14 14 0 14.0 0.0 -14.0 1113.742 0.0 -14.0 0.0 -10.175 14.0 -1.8771428571428572 14.0 0.0 14 -NULL -35 1969-12-31 16:00:05.143 -35.0 NULL 35 -35 0 -35.0 -0.0 35.0 -2784.355 0.0 35.0 0.0 -10.175 -35.0 0.7508571428571429 -35.0 0.0 -35 -true -20 1969-12-31 16:00:08.986 -20.0 cvLH6Eat2yFsyy7p 20 -20 0 -20.0 -0.0 20.0 -1591.06 0.0 20.0 0.0 -10.175 -20.0 1.314 -20.0 0.0 -20 -NULL -8 1969-12-31 16:00:04.443 -8.0 NULL 8 -8 0 -8.0 -0.0 8.0 -636.424 0.0 8.0 0.0 -10.175 -8.0 3.285 -8.0 0.0 -8 -true -5 1969-12-31 16:00:00.611 -5.0 cvLH6Eat2yFsyy7p 5 -5 0 -5.0 -0.0 5.0 -397.765 0.0 5.0 0.0 -10.175 -5.0 5.256 -5.0 0.0 -5 -NULL 39 1969-12-31 16:00:14.79 39.0 NULL -39 39 0 39.0 0.0 -39.0 3102.567 0.0 -39.0 0.0 -10.175 39.0 -0.6738461538461539 39.0 0.0 39 -NULL -25 1969-12-31 16:00:09.682 -25.0 NULL 25 -25 0 -25.0 -0.0 25.0 -1988.8249999999998 0.0 25.0 0.0 -10.175 -25.0 1.0512000000000001 -25.0 0.0 -25 -NULL 54 1969-12-31 16:00:13.128 54.0 NULL -54 54 0 54.0 0.0 -54.0 4295.862 0.0 -54.0 0.0 -10.175 54.0 -0.4866666666666667 54.0 0.0 54 -NULL -27 1969-12-31 16:00:08.608 -27.0 NULL 27 -27 0 -27.0 -0.0 27.0 -2147.931 0.0 27.0 0.0 -10.175 -27.0 0.9733333333333334 -27.0 0.0 -27 -true -14 1969-12-31 16:00:14.609 -14.0 821UdmGbkEf4j 14 -14 0 -14.0 -0.0 14.0 -1113.742 0.0 14.0 0.0 -10.175 -14.0 1.8771428571428572 -14.0 0.0 -14 -true -32 1969-12-31 16:00:15.62 -32.0 821UdmGbkEf4j 32 -32 0 -32.0 -0.0 32.0 -2545.696 0.0 32.0 0.0 -10.175 -32.0 0.82125 -32.0 0.0 -32 -true -58 1969-12-31 16:00:02.404 -58.0 821UdmGbkEf4j 58 -58 0 -58.0 -0.0 58.0 -4614.074 0.0 58.0 0.0 -10.175 -58.0 0.4531034482758621 -58.0 0.0 -58 -true 15 1969-12-31 16:00:01.115 15.0 1cGVWH7n1QU -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -true 53 1969-12-31 16:00:03.259 53.0 1cGVWH7n1QU -53 53 0 53.0 0.0 -53.0 4216.309 0.0 -53.0 0.0 -10.175 53.0 -0.4958490566037736 53.0 0.0 53 -NULL 46 1969-12-31 16:00:09.883 46.0 NULL -46 46 0 46.0 0.0 -46.0 3659.438 0.0 -46.0 0.0 -10.175 46.0 -0.571304347826087 46.0 0.0 46 -NULL 13 1969-12-31 16:00:06.749 13.0 NULL -13 13 0 13.0 0.0 -13.0 1034.1889999999999 0.0 -13.0 0.0 -10.175 13.0 -2.0215384615384617 13.0 0.0 13 -true -43 1969-12-31 16:00:12.417 -43.0 821UdmGbkEf4j 43 -43 0 -43.0 -0.0 43.0 -3420.779 0.0 43.0 0.0 -10.175 -43.0 0.6111627906976744 -43.0 0.0 -43 -NULL -17 1969-12-31 16:00:10.104 -17.0 NULL 17 -17 0 -17.0 -0.0 17.0 -1352.4009999999998 0.0 17.0 0.0 -10.175 -17.0 1.5458823529411765 -17.0 0.0 -17 -NULL 43 1969-12-31 16:00:01.097 43.0 NULL -43 43 0 43.0 0.0 -43.0 3420.779 0.0 -43.0 0.0 -10.175 43.0 -0.6111627906976744 43.0 0.0 43 -true -23 1969-12-31 16:00:09.177 -23.0 1cGVWH7n1QU 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -true 4 1969-12-31 16:00:00.237 4.0 821UdmGbkEf4j -4 4 0 4.0 0.0 -4.0 318.212 0.0 -4.0 0.0 -10.175 4.0 -6.57 4.0 0.0 4 -NULL 25 1969-12-31 16:00:06.03 25.0 NULL -25 25 0 25.0 0.0 -25.0 1988.8249999999998 0.0 -25.0 0.0 -10.175 25.0 -1.0512000000000001 25.0 0.0 25 -NULL -3 1969-12-31 16:00:01.723 -3.0 NULL 3 -3 0 -3.0 -0.0 3.0 -238.659 0.0 3.0 0.0 -10.175 -3.0 8.76 -3.0 0.0 -3 -true -23 1969-12-31 16:00:09.011 -23.0 821UdmGbkEf4j 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -NULL -5 1969-12-31 16:00:02.021 -5.0 NULL 5 -5 0 -5.0 -0.0 5.0 -397.765 0.0 5.0 0.0 -10.175 -5.0 5.256 -5.0 0.0 -5 -true -8 1969-12-31 16:00:06.513 -8.0 821UdmGbkEf4j 8 -8 0 -8.0 -0.0 8.0 -636.424 0.0 8.0 0.0 -10.175 -8.0 3.285 -8.0 0.0 -8 -NULL -39 1969-12-31 16:00:08.852 -39.0 NULL 39 -39 0 -39.0 -0.0 39.0 -3102.567 0.0 39.0 0.0 -10.175 -39.0 0.6738461538461539 -39.0 0.0 -39 -NULL -20 1969-12-31 16:00:13.778 -20.0 NULL 20 -20 0 -20.0 -0.0 20.0 -1591.06 0.0 20.0 0.0 -10.175 -20.0 1.314 -20.0 0.0 -20 -true -38 1969-12-31 16:00:09.246 -38.0 1cGVWH7n1QU 38 -38 0 -38.0 -0.0 38.0 -3023.014 0.0 38.0 0.0 -10.175 -38.0 0.6915789473684211 -38.0 0.0 -38 -NULL 37 1969-12-31 16:00:14.674 37.0 NULL -37 37 0 37.0 0.0 -37.0 2943.461 0.0 -37.0 0.0 -10.175 37.0 -0.7102702702702703 37.0 0.0 37 -true -10 1969-12-31 16:00:08.523 -10.0 cvLH6Eat2yFsyy7p 10 -10 0 -10.0 -0.0 10.0 -795.53 0.0 10.0 0.0 -10.175 -10.0 2.628 -10.0 0.0 -10 -true 33 1969-12-31 16:00:03.208 33.0 cvLH6Eat2yFsyy7p -33 33 0 33.0 0.0 -33.0 2625.249 0.0 -33.0 0.0 -10.175 33.0 -0.7963636363636364 33.0 0.0 33 -NULL 40 1969-12-31 16:00:00.781 40.0 NULL -40 40 0 40.0 0.0 -40.0 3182.12 0.0 -40.0 0.0 -10.175 40.0 -0.657 40.0 0.0 40 -NULL 1 1969-12-31 16:00:04.916 1.0 NULL -1 1 0 1.0 0.0 -1.0 79.553 0.0 -1.0 0.0 -10.175 1.0 -26.28 1.0 0.0 1 -NULL -46 1969-12-31 16:00:00.025 -46.0 NULL 46 -46 0 -46.0 -0.0 46.0 -3659.438 0.0 46.0 0.0 -10.175 -46.0 0.571304347826087 -46.0 0.0 -46 -true -41 1969-12-31 16:00:11.132 -41.0 1cGVWH7n1QU 41 -41 0 -41.0 -0.0 41.0 -3261.673 0.0 41.0 0.0 -10.175 -41.0 0.6409756097560976 -41.0 0.0 -41 -true -16 1969-12-31 16:00:09.246 -16.0 1cGVWH7n1QU 16 -16 0 -16.0 -0.0 16.0 -1272.848 0.0 16.0 0.0 -10.175 -16.0 1.6425 -16.0 0.0 -16 -NULL 38 1969-12-31 16:00:16.279 38.0 NULL -38 38 0 38.0 0.0 -38.0 3023.014 0.0 -38.0 0.0 -10.175 38.0 -0.6915789473684211 38.0 0.0 38 -NULL 19 1969-12-31 16:00:01.375 19.0 NULL -19 19 0 19.0 0.0 -19.0 1511.507 0.0 -19.0 0.0 -10.175 19.0 -1.3831578947368421 19.0 0.0 19 -NULL 39 1969-12-31 16:00:05.707 39.0 NULL -39 39 0 39.0 0.0 -39.0 3102.567 0.0 -39.0 0.0 -10.175 39.0 -0.6738461538461539 39.0 0.0 39 -true -45 1969-12-31 16:00:04.832 -45.0 cvLH6Eat2yFsyy7p 45 -45 0 -45.0 -0.0 45.0 -3579.8849999999998 0.0 45.0 0.0 -10.175 -45.0 0.5840000000000001 -45.0 0.0 -45 -true 36 1969-12-31 16:00:14.347 36.0 821UdmGbkEf4j -36 36 0 36.0 0.0 -36.0 2863.908 0.0 -36.0 0.0 -10.175 36.0 -0.73 36.0 0.0 36 -NULL 33 1969-12-31 16:00:07.421 33.0 NULL -33 33 0 33.0 0.0 -33.0 2625.249 0.0 -33.0 0.0 -10.175 33.0 -0.7963636363636364 33.0 0.0 33 -true 39 1969-12-31 16:00:10.12 39.0 cvLH6Eat2yFsyy7p -39 39 0 39.0 0.0 -39.0 3102.567 0.0 -39.0 0.0 -10.175 39.0 -0.6738461538461539 39.0 0.0 39 -NULL -21 1969-12-31 16:00:12.544 -21.0 NULL 21 -21 0 -21.0 -0.0 21.0 -1670.6129999999998 0.0 21.0 0.0 -10.175 -21.0 1.2514285714285716 -21.0 0.0 -21 -true -59 1969-12-31 16:00:11.065 -59.0 821UdmGbkEf4j 59 -59 0 -59.0 -0.0 59.0 -4693.6269999999995 0.0 59.0 0.0 -10.175 -59.0 0.44542372881355935 -59.0 0.0 -59 -NULL 11 1969-12-31 16:00:08.745 11.0 NULL -11 11 0 11.0 0.0 -11.0 875.083 0.0 -11.0 0.0 -10.175 11.0 -2.389090909090909 11.0 0.0 11 -NULL 42 1969-12-31 16:00:11.758 42.0 NULL -42 42 0 42.0 0.0 -42.0 3341.2259999999997 0.0 -42.0 0.0 -10.175 42.0 -0.6257142857142858 42.0 0.0 42 -NULL -54 1969-12-31 16:00:05.688 -54.0 NULL 54 -54 0 -54.0 -0.0 54.0 -4295.862 0.0 54.0 0.0 -10.175 -54.0 0.4866666666666667 -54.0 0.0 -54 -NULL 25 1969-12-31 16:00:10.14 25.0 NULL -25 25 0 25.0 0.0 -25.0 1988.8249999999998 0.0 -25.0 0.0 -10.175 25.0 -1.0512000000000001 25.0 0.0 25 -NULL 19 1969-12-31 16:00:10.711 19.0 NULL -19 19 0 19.0 0.0 -19.0 1511.507 0.0 -19.0 0.0 -10.175 19.0 -1.3831578947368421 19.0 0.0 19 -true 56 1969-12-31 16:00:04.286 56.0 1cGVWH7n1QU -56 56 0 56.0 0.0 -56.0 4454.968 0.0 -56.0 0.0 -10.175 56.0 -0.4692857142857143 56.0 0.0 56 -NULL -10 1969-12-31 16:00:00.916 -10.0 NULL 10 -10 0 -10.0 -0.0 10.0 -795.53 0.0 10.0 0.0 -10.175 -10.0 2.628 -10.0 0.0 -10 -true 43 1969-12-31 16:00:07.435 43.0 1cGVWH7n1QU -43 43 0 43.0 0.0 -43.0 3420.779 0.0 -43.0 0.0 -10.175 43.0 -0.6111627906976744 43.0 0.0 43 -true -19 1969-12-31 16:00:00.348 -19.0 1cGVWH7n1QU 19 -19 0 -19.0 -0.0 19.0 -1511.507 0.0 19.0 0.0 -10.175 -19.0 1.3831578947368421 -19.0 0.0 -19 -true 10 1969-12-31 16:00:01.358 10.0 cvLH6Eat2yFsyy7p -10 10 0 10.0 0.0 -10.0 795.53 0.0 -10.0 0.0 -10.175 10.0 -2.628 10.0 0.0 10 -true 11 1969-12-31 16:00:02.521 11.0 1cGVWH7n1QU -11 11 0 11.0 0.0 -11.0 875.083 0.0 -11.0 0.0 -10.175 11.0 -2.389090909090909 11.0 0.0 11 -true 0 1969-12-31 16:00:04.336 0.0 cvLH6Eat2yFsyy7p 0 0 0 0.0 0.0 -0.0 0.0 0.0 -0.0 0.0 -10.175 0.0 NULL 0.0 NULL 0 -NULL 62 1969-12-31 16:00:00.461 62.0 NULL -62 62 0 62.0 0.0 -62.0 4932.286 0.0 -62.0 0.0 -10.175 62.0 -0.4238709677419355 62.0 0.0 62 -NULL -51 1969-12-31 16:00:02.011 -51.0 NULL 51 -51 0 -51.0 -0.0 51.0 -4057.203 0.0 51.0 0.0 -10.175 -51.0 0.5152941176470588 -51.0 0.0 -51 -NULL 5 1969-12-31 16:00:04.259 5.0 NULL -5 5 0 5.0 0.0 -5.0 397.765 0.0 -5.0 0.0 -10.175 5.0 -5.256 5.0 0.0 5 -true -47 1969-12-31 16:00:04.811 -47.0 cvLH6Eat2yFsyy7p 47 -47 0 -47.0 -0.0 47.0 -3738.991 0.0 47.0 0.0 -10.175 -47.0 0.5591489361702128 -47.0 0.0 -47 -true 35 1969-12-31 16:00:03.799 35.0 821UdmGbkEf4j -35 35 0 35.0 0.0 -35.0 2784.355 0.0 -35.0 0.0 -10.175 35.0 -0.7508571428571429 35.0 0.0 35 -NULL 57 1969-12-31 16:00:06.997 57.0 NULL -57 57 0 57.0 0.0 -57.0 4534.521 0.0 -57.0 0.0 -10.175 57.0 -0.4610526315789474 57.0 0.0 57 -NULL -6 1969-12-31 16:00:08.894 -6.0 NULL 6 -6 0 -6.0 -0.0 6.0 -477.318 0.0 6.0 0.0 -10.175 -6.0 4.38 -6.0 0.0 -6 -NULL 29 1969-12-31 16:00:02.777 29.0 NULL -29 29 0 29.0 0.0 -29.0 2307.037 0.0 -29.0 0.0 -10.175 29.0 -0.9062068965517242 29.0 0.0 29 -NULL 6 1969-12-31 16:00:11.364 6.0 NULL -6 6 0 6.0 0.0 -6.0 477.318 0.0 -6.0 0.0 -10.175 6.0 -4.38 6.0 0.0 6 -true 31 1969-12-31 16:00:04.579 31.0 cvLH6Eat2yFsyy7p -31 31 0 31.0 0.0 -31.0 2466.143 0.0 -31.0 0.0 -10.175 31.0 -0.847741935483871 31.0 0.0 31 -NULL -19 1969-12-31 16:00:01.714 -19.0 NULL 19 -19 0 -19.0 -0.0 19.0 -1511.507 0.0 19.0 0.0 -10.175 -19.0 1.3831578947368421 -19.0 0.0 -19 -true -61 1969-12-31 16:00:15.694 -61.0 cvLH6Eat2yFsyy7p 61 -61 0 -61.0 -0.0 61.0 -4852.733 0.0 61.0 0.0 -10.175 -61.0 0.4308196721311476 -61.0 0.0 -61 -true -16 1969-12-31 16:00:03.867 -16.0 1cGVWH7n1QU 16 -16 0 -16.0 -0.0 16.0 -1272.848 0.0 16.0 0.0 -10.175 -16.0 1.6425 -16.0 0.0 -16 -true 14 1969-12-31 16:00:15.709 14.0 1cGVWH7n1QU -14 14 0 14.0 0.0 -14.0 1113.742 0.0 -14.0 0.0 -10.175 14.0 -1.8771428571428572 14.0 0.0 14 -true 19 1969-12-31 16:00:14.479 19.0 1cGVWH7n1QU -19 19 0 19.0 0.0 -19.0 1511.507 0.0 -19.0 0.0 -10.175 19.0 -1.3831578947368421 19.0 0.0 19 -true 22 1969-12-31 16:00:03.815 22.0 821UdmGbkEf4j -22 22 0 22.0 0.0 -22.0 1750.166 0.0 -22.0 0.0 -10.175 22.0 -1.1945454545454546 22.0 0.0 22 -NULL -33 1969-12-31 16:00:02.752 -33.0 NULL 33 -33 0 -33.0 -0.0 33.0 -2625.249 0.0 33.0 0.0 -10.175 -33.0 0.7963636363636364 -33.0 0.0 -33 -NULL -28 1969-12-31 16:00:16.192 -28.0 NULL 28 -28 0 -28.0 -0.0 28.0 -2227.484 0.0 28.0 0.0 -10.175 -28.0 0.9385714285714286 -28.0 0.0 -28 -true 30 1969-12-31 16:00:16.282 30.0 cvLH6Eat2yFsyy7p -30 30 0 30.0 0.0 -30.0 2386.59 0.0 -30.0 0.0 -10.175 30.0 -0.876 30.0 0.0 30 -true -20 1969-12-31 16:00:13.823 -20.0 1cGVWH7n1QU 20 -20 0 -20.0 -0.0 20.0 -1591.06 0.0 20.0 0.0 -10.175 -20.0 1.314 -20.0 0.0 -20 -true -48 1969-12-31 16:00:00.645 -48.0 cvLH6Eat2yFsyy7p 48 -48 0 -48.0 -0.0 48.0 -3818.544 0.0 48.0 0.0 -10.175 -48.0 0.5475 -48.0 0.0 -48 -true -33 1969-12-31 16:00:03.018 -33.0 821UdmGbkEf4j 33 -33 0 -33.0 -0.0 33.0 -2625.249 0.0 33.0 0.0 -10.175 -33.0 0.7963636363636364 -33.0 0.0 -33 -true 51 1969-12-31 16:00:07.406 51.0 1cGVWH7n1QU -51 51 0 51.0 0.0 -51.0 4057.203 0.0 -51.0 0.0 -10.175 51.0 -0.5152941176470588 51.0 0.0 51 -true 42 1969-12-31 16:00:14.111 42.0 821UdmGbkEf4j -42 42 0 42.0 0.0 -42.0 3341.2259999999997 0.0 -42.0 0.0 -10.175 42.0 -0.6257142857142858 42.0 0.0 42 -NULL -41 1969-12-31 16:00:02.969 -41.0 NULL 41 -41 0 -41.0 -0.0 41.0 -3261.673 0.0 41.0 0.0 -10.175 -41.0 0.6409756097560976 -41.0 0.0 -41 -true 56 1969-12-31 16:00:14.785 56.0 821UdmGbkEf4j -56 56 0 56.0 0.0 -56.0 4454.968 0.0 -56.0 0.0 -10.175 56.0 -0.4692857142857143 56.0 0.0 56 -true -60 1969-12-31 16:00:08.445 -60.0 cvLH6Eat2yFsyy7p 60 -60 0 -60.0 -0.0 60.0 -4773.18 0.0 60.0 0.0 -10.175 -60.0 0.438 -60.0 0.0 -60 -true 39 1969-12-31 16:00:10.403 39.0 821UdmGbkEf4j -39 39 0 39.0 0.0 -39.0 3102.567 0.0 -39.0 0.0 -10.175 39.0 -0.6738461538461539 39.0 0.0 39 -true -34 1969-12-31 16:00:13.249 -34.0 1cGVWH7n1QU 34 -34 0 -34.0 -0.0 34.0 -2704.8019999999997 0.0 34.0 0.0 -10.175 -34.0 0.7729411764705882 -34.0 0.0 -34 -NULL 2 1969-12-31 16:00:06.982 2.0 NULL -2 2 0 2.0 0.0 -2.0 159.106 0.0 -2.0 0.0 -10.175 2.0 -13.14 2.0 0.0 2 -NULL -11 1969-12-31 16:00:13.15 -11.0 NULL 11 -11 0 -11.0 -0.0 11.0 -875.083 0.0 11.0 0.0 -10.175 -11.0 2.389090909090909 -11.0 0.0 -11 -NULL -57 1969-12-31 16:00:05.5 -57.0 NULL 57 -57 0 -57.0 -0.0 57.0 -4534.521 0.0 57.0 0.0 -10.175 -57.0 0.4610526315789474 -57.0 0.0 -57 -NULL 48 1969-12-31 16:00:15.548 48.0 NULL -48 48 0 48.0 0.0 -48.0 3818.544 0.0 -48.0 0.0 -10.175 48.0 -0.5475 48.0 0.0 48 -true 53 1969-12-31 16:00:02.611 53.0 821UdmGbkEf4j -53 53 0 53.0 0.0 -53.0 4216.309 0.0 -53.0 0.0 -10.175 53.0 -0.4958490566037736 53.0 0.0 53 -true 8 1969-12-31 16:00:06.327 8.0 cvLH6Eat2yFsyy7p -8 8 0 8.0 0.0 -8.0 636.424 0.0 -8.0 0.0 -10.175 8.0 -3.285 8.0 0.0 8 -true 24 1969-12-31 16:00:11.359 24.0 821UdmGbkEf4j -24 24 0 24.0 0.0 -24.0 1909.272 0.0 -24.0 0.0 -10.175 24.0 -1.095 24.0 0.0 24 -NULL 43 1969-12-31 16:00:08.554 43.0 NULL -43 43 0 43.0 0.0 -43.0 3420.779 0.0 -43.0 0.0 -10.175 43.0 -0.6111627906976744 43.0 0.0 43 -true -38 1969-12-31 16:00:06.823 -38.0 821UdmGbkEf4j 38 -38 0 -38.0 -0.0 38.0 -3023.014 0.0 38.0 0.0 -10.175 -38.0 0.6915789473684211 -38.0 0.0 -38 -true -16 1969-12-31 16:00:14.172 -16.0 821UdmGbkEf4j 16 -16 0 -16.0 -0.0 16.0 -1272.848 0.0 16.0 0.0 -10.175 -16.0 1.6425 -16.0 0.0 -16 -NULL 13 1969-12-31 16:00:03.794 13.0 NULL -13 13 0 13.0 0.0 -13.0 1034.1889999999999 0.0 -13.0 0.0 -10.175 13.0 -2.0215384615384617 13.0 0.0 13 -true -21 1969-12-31 16:00:14.783 -21.0 1cGVWH7n1QU 21 -21 0 -21.0 -0.0 21.0 -1670.6129999999998 0.0 21.0 0.0 -10.175 -21.0 1.2514285714285716 -21.0 0.0 -21 -true -51 1969-12-31 16:00:02.479 -51.0 cvLH6Eat2yFsyy7p 51 -51 0 -51.0 -0.0 51.0 -4057.203 0.0 51.0 0.0 -10.175 -51.0 0.5152941176470588 -51.0 0.0 -51 -true 8 1969-12-31 16:00:10.496 8.0 821UdmGbkEf4j -8 8 0 8.0 0.0 -8.0 636.424 0.0 -8.0 0.0 -10.175 8.0 -3.285 8.0 0.0 8 -true -1 1969-12-31 16:00:00.519 -1.0 cvLH6Eat2yFsyy7p 1 -1 0 -1.0 -0.0 1.0 -79.553 0.0 1.0 0.0 -10.175 -1.0 26.28 -1.0 0.0 -1 -true -38 1969-12-31 16:00:08.084 -38.0 821UdmGbkEf4j 38 -38 0 -38.0 -0.0 38.0 -3023.014 0.0 38.0 0.0 -10.175 -38.0 0.6915789473684211 -38.0 0.0 -38 -NULL -59 1969-12-31 16:00:00.831 -59.0 NULL 59 -59 0 -59.0 -0.0 59.0 -4693.6269999999995 0.0 59.0 0.0 -10.175 -59.0 0.44542372881355935 -59.0 0.0 -59 -true 57 1969-12-31 16:00:08.837 57.0 821UdmGbkEf4j -57 57 0 57.0 0.0 -57.0 4534.521 0.0 -57.0 0.0 -10.175 57.0 -0.4610526315789474 57.0 0.0 57 -true -28 1969-12-31 16:00:05.042 -28.0 cvLH6Eat2yFsyy7p 28 -28 0 -28.0 -0.0 28.0 -2227.484 0.0 28.0 0.0 -10.175 -28.0 0.9385714285714286 -28.0 0.0 -28 -NULL 33 1969-12-31 16:00:07.006 33.0 NULL -33 33 0 33.0 0.0 -33.0 2625.249 0.0 -33.0 0.0 -10.175 33.0 -0.7963636363636364 33.0 0.0 33 -true -57 1969-12-31 16:00:05.19 -57.0 821UdmGbkEf4j 57 -57 0 -57.0 -0.0 57.0 -4534.521 0.0 57.0 0.0 -10.175 -57.0 0.4610526315789474 -57.0 0.0 -57 -true -58 1969-12-31 16:00:00.027 -58.0 1cGVWH7n1QU 58 -58 0 -58.0 -0.0 58.0 -4614.074 0.0 58.0 0.0 -10.175 -58.0 0.4531034482758621 -58.0 0.0 -58 -true -61 1969-12-31 16:00:12.75 -61.0 cvLH6Eat2yFsyy7p 61 -61 0 -61.0 -0.0 61.0 -4852.733 0.0 61.0 0.0 -10.175 -61.0 0.4308196721311476 -61.0 0.0 -61 -true -2 1969-12-31 16:00:11.979 -2.0 cvLH6Eat2yFsyy7p 2 -2 0 -2.0 -0.0 2.0 -159.106 0.0 2.0 0.0 -10.175 -2.0 13.14 -2.0 0.0 -2 -NULL -34 1969-12-31 16:00:04.756 -34.0 NULL 34 -34 0 -34.0 -0.0 34.0 -2704.8019999999997 0.0 34.0 0.0 -10.175 -34.0 0.7729411764705882 -34.0 0.0 -34 -true 0 1969-12-31 16:00:07.552 0.0 cvLH6Eat2yFsyy7p 0 0 0 0.0 0.0 -0.0 0.0 0.0 -0.0 0.0 -10.175 0.0 NULL 0.0 NULL 0 -NULL 32 1969-12-31 16:00:03.571 32.0 NULL -32 32 0 32.0 0.0 -32.0 2545.696 0.0 -32.0 0.0 -10.175 32.0 -0.82125 32.0 0.0 32 -NULL -30 1969-12-31 16:00:11.91 -30.0 NULL 30 -30 0 -30.0 -0.0 30.0 -2386.59 0.0 30.0 0.0 -10.175 -30.0 0.876 -30.0 0.0 -30 -true -23 1969-12-31 16:00:13.984 -23.0 1cGVWH7n1QU 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -true -15 1969-12-31 16:00:13.192 -15.0 cvLH6Eat2yFsyy7p 15 -15 0 -15.0 -0.0 15.0 -1193.295 0.0 15.0 0.0 -10.175 -15.0 1.752 -15.0 0.0 -15 -true 44 1969-12-31 16:00:11.935 44.0 cvLH6Eat2yFsyy7p -44 44 0 44.0 0.0 -44.0 3500.332 0.0 -44.0 0.0 -10.175 44.0 -0.5972727272727273 44.0 0.0 44 -NULL 14 1969-12-31 16:00:12.205 14.0 NULL -14 14 0 14.0 0.0 -14.0 1113.742 0.0 -14.0 0.0 -10.175 14.0 -1.8771428571428572 14.0 0.0 14 -true 6 1969-12-31 16:00:13.987 6.0 cvLH6Eat2yFsyy7p -6 6 0 6.0 0.0 -6.0 477.318 0.0 -6.0 0.0 -10.175 6.0 -4.38 6.0 0.0 6 -true -44 1969-12-31 16:00:06.76 -44.0 821UdmGbkEf4j 44 -44 0 -44.0 -0.0 44.0 -3500.332 0.0 44.0 0.0 -10.175 -44.0 0.5972727272727273 -44.0 0.0 -44 -true 60 1969-12-31 16:00:04.488 60.0 821UdmGbkEf4j -60 60 0 60.0 0.0 -60.0 4773.18 0.0 -60.0 0.0 -10.175 60.0 -0.438 60.0 0.0 60 -true -1 1969-12-31 16:00:06.036 -1.0 821UdmGbkEf4j 1 -1 0 -1.0 -0.0 1.0 -79.553 0.0 1.0 0.0 -10.175 -1.0 26.28 -1.0 0.0 -1 -NULL -12 1969-12-31 16:00:09.524 -12.0 NULL 12 -12 0 -12.0 -0.0 12.0 -954.636 0.0 12.0 0.0 -10.175 -12.0 2.19 -12.0 0.0 -12 -NULL 49 1969-12-31 16:00:08.726 49.0 NULL -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 -true -17 1969-12-31 16:00:08.072 -17.0 821UdmGbkEf4j 17 -17 0 -17.0 -0.0 17.0 -1352.4009999999998 0.0 17.0 0.0 -10.175 -17.0 1.5458823529411765 -17.0 0.0 -17 -true -47 1969-12-31 16:00:00.702 -47.0 1cGVWH7n1QU 47 -47 0 -47.0 -0.0 47.0 -3738.991 0.0 47.0 0.0 -10.175 -47.0 0.5591489361702128 -47.0 0.0 -47 -NULL 10 1969-12-31 16:00:12.529 10.0 NULL -10 10 0 10.0 0.0 -10.0 795.53 0.0 -10.0 0.0 -10.175 10.0 -2.628 10.0 0.0 10 -NULL 0 1969-12-31 16:00:15.616 0.0 NULL 0 0 0 0.0 0.0 -0.0 0.0 0.0 -0.0 0.0 -10.175 0.0 NULL 0.0 NULL 0 -true -8 1969-12-31 16:00:00.511 -8.0 1cGVWH7n1QU 8 -8 0 -8.0 -0.0 8.0 -636.424 0.0 8.0 0.0 -10.175 -8.0 3.285 -8.0 0.0 -8 -NULL -39 1969-12-31 16:00:00.708 -39.0 NULL 39 -39 0 -39.0 -0.0 39.0 -3102.567 0.0 39.0 0.0 -10.175 -39.0 0.6738461538461539 -39.0 0.0 -39 -NULL 57 1969-12-31 16:00:04.254 57.0 NULL -57 57 0 57.0 0.0 -57.0 4534.521 0.0 -57.0 0.0 -10.175 57.0 -0.4610526315789474 57.0 0.0 57 -true -16 1969-12-31 16:00:13.51 -16.0 1cGVWH7n1QU 16 -16 0 -16.0 -0.0 16.0 -1272.848 0.0 16.0 0.0 -10.175 -16.0 1.6425 -16.0 0.0 -16 -NULL -52 1969-12-31 16:00:04.518 -52.0 NULL 52 -52 0 -52.0 -0.0 52.0 -4136.755999999999 0.0 52.0 0.0 -10.175 -52.0 0.5053846153846154 -52.0 0.0 -52 -true 46 1969-12-31 16:00:04.245 46.0 cvLH6Eat2yFsyy7p -46 46 0 46.0 0.0 -46.0 3659.438 0.0 -46.0 0.0 -10.175 46.0 -0.571304347826087 46.0 0.0 46 -NULL 4 1969-12-31 16:00:04.735 4.0 NULL -4 4 0 4.0 0.0 -4.0 318.212 0.0 -4.0 0.0 -10.175 4.0 -6.57 4.0 0.0 4 -NULL -38 1969-12-31 16:00:07.008 -38.0 NULL 38 -38 0 -38.0 -0.0 38.0 -3023.014 0.0 38.0 0.0 -10.175 -38.0 0.6915789473684211 -38.0 0.0 -38 -NULL -9 1969-12-31 16:00:10.227 -9.0 NULL 9 -9 0 -9.0 -0.0 9.0 -715.977 0.0 9.0 0.0 -10.175 -9.0 2.92 -9.0 0.0 -9 -true 10 1969-12-31 16:00:09.539 10.0 821UdmGbkEf4j -10 10 0 10.0 0.0 -10.0 795.53 0.0 -10.0 0.0 -10.175 10.0 -2.628 10.0 0.0 10 -true 3 1969-12-31 16:00:10.098 3.0 1cGVWH7n1QU -3 3 0 3.0 0.0 -3.0 238.659 0.0 -3.0 0.0 -10.175 3.0 -8.76 3.0 0.0 3 -true 2 1969-12-31 16:00:01.703 2.0 821UdmGbkEf4j -2 2 0 2.0 0.0 -2.0 159.106 0.0 -2.0 0.0 -10.175 2.0 -13.14 2.0 0.0 2 -NULL 5 1969-12-31 16:00:09.903 5.0 NULL -5 5 0 5.0 0.0 -5.0 397.765 0.0 -5.0 0.0 -10.175 5.0 -5.256 5.0 0.0 5 -NULL -54 1969-12-31 16:00:02.859 -54.0 NULL 54 -54 0 -54.0 -0.0 54.0 -4295.862 0.0 54.0 0.0 -10.175 -54.0 0.4866666666666667 -54.0 0.0 -54 -NULL -33 1969-12-31 16:00:08.046 -33.0 NULL 33 -33 0 -33.0 -0.0 33.0 -2625.249 0.0 33.0 0.0 -10.175 -33.0 0.7963636363636364 -33.0 0.0 -33 -true -25 1969-12-31 16:00:00.041 -25.0 cvLH6Eat2yFsyy7p 25 -25 0 -25.0 -0.0 25.0 -1988.8249999999998 0.0 25.0 0.0 -10.175 -25.0 1.0512000000000001 -25.0 0.0 -25 -true 27 1969-12-31 16:00:09.243 27.0 cvLH6Eat2yFsyy7p -27 27 0 27.0 0.0 -27.0 2147.931 0.0 -27.0 0.0 -10.175 27.0 -0.9733333333333334 27.0 0.0 27 -NULL 0 1969-12-31 16:00:09.544 0.0 NULL 0 0 0 0.0 0.0 -0.0 0.0 0.0 -0.0 0.0 -10.175 0.0 NULL 0.0 NULL 0 -NULL 26 1969-12-31 16:00:05.219 26.0 NULL -26 26 0 26.0 0.0 -26.0 2068.3779999999997 0.0 -26.0 0.0 -10.175 26.0 -1.0107692307692309 26.0 0.0 26 -true 15 1969-12-31 16:00:14.822 15.0 821UdmGbkEf4j -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -NULL -41 1969-12-31 16:00:07.204 -41.0 NULL 41 -41 0 -41.0 -0.0 41.0 -3261.673 0.0 41.0 0.0 -10.175 -41.0 0.6409756097560976 -41.0 0.0 -41 -NULL 22 1969-12-31 16:00:05.731 22.0 NULL -22 22 0 22.0 0.0 -22.0 1750.166 0.0 -22.0 0.0 -10.175 22.0 -1.1945454545454546 22.0 0.0 22 -NULL 41 1969-12-31 16:00:03.512 41.0 NULL -41 41 0 41.0 0.0 -41.0 3261.673 0.0 -41.0 0.0 -10.175 41.0 -0.6409756097560976 41.0 0.0 41 -NULL -35 1969-12-31 16:00:04.52 -35.0 NULL 35 -35 0 -35.0 -0.0 35.0 -2784.355 0.0 35.0 0.0 -10.175 -35.0 0.7508571428571429 -35.0 0.0 -35 -NULL 49 1969-12-31 16:00:05.369 49.0 NULL -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 -true 7 1969-12-31 16:00:15.998 7.0 1cGVWH7n1QU -7 7 0 7.0 0.0 -7.0 556.871 0.0 -7.0 0.0 -10.175 7.0 -3.7542857142857144 7.0 0.0 7 -true -35 1969-12-31 16:00:01.435 -35.0 821UdmGbkEf4j 35 -35 0 -35.0 -0.0 35.0 -2784.355 0.0 35.0 0.0 -10.175 -35.0 0.7508571428571429 -35.0 0.0 -35 -NULL 37 1969-12-31 16:00:08.692 37.0 NULL -37 37 0 37.0 0.0 -37.0 2943.461 0.0 -37.0 0.0 -10.175 37.0 -0.7102702702702703 37.0 0.0 37 -true 9 1969-12-31 16:00:13.387 9.0 cvLH6Eat2yFsyy7p -9 9 0 9.0 0.0 -9.0 715.977 0.0 -9.0 0.0 -10.175 9.0 -2.92 9.0 0.0 9 -NULL 15 1969-12-31 16:00:11.485 15.0 NULL -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -NULL 23 1969-12-31 16:00:00.315 23.0 NULL -23 23 0 23.0 0.0 -23.0 1829.719 0.0 -23.0 0.0 -10.175 23.0 -1.142608695652174 23.0 0.0 23 -NULL 8 1969-12-31 16:00:09.995 8.0 NULL -8 8 0 8.0 0.0 -8.0 636.424 0.0 -8.0 0.0 -10.175 8.0 -3.285 8.0 0.0 8 -true 22 1969-12-31 16:00:03.555 22.0 821UdmGbkEf4j -22 22 0 22.0 0.0 -22.0 1750.166 0.0 -22.0 0.0 -10.175 22.0 -1.1945454545454546 22.0 0.0 22 -true -36 1969-12-31 16:00:15.112 -36.0 cvLH6Eat2yFsyy7p 36 -36 0 -36.0 -0.0 36.0 -2863.908 0.0 36.0 0.0 -10.175 -36.0 0.73 -36.0 0.0 -36 -NULL -63 1969-12-31 16:00:15.436 -63.0 NULL 63 -63 0 -63.0 -0.0 63.0 -5011.839 0.0 63.0 0.0 -10.175 -63.0 0.41714285714285715 -63.0 0.0 -63 -NULL -4 1969-12-31 16:00:13.324 -4.0 NULL 4 -4 0 -4.0 -0.0 4.0 -318.212 0.0 4.0 0.0 -10.175 -4.0 6.57 -4.0 0.0 -4 -NULL 37 1969-12-31 16:00:12.728 37.0 NULL -37 37 0 37.0 0.0 -37.0 2943.461 0.0 -37.0 0.0 -10.175 37.0 -0.7102702702702703 37.0 0.0 37 -true 0 1969-12-31 16:00:00.181 0.0 cvLH6Eat2yFsyy7p 0 0 0 0.0 0.0 -0.0 0.0 0.0 -0.0 0.0 -10.175 0.0 NULL 0.0 NULL 0 -true -8 1969-12-31 16:00:07.588 -8.0 821UdmGbkEf4j 8 -8 0 -8.0 -0.0 8.0 -636.424 0.0 8.0 0.0 -10.175 -8.0 3.285 -8.0 0.0 -8 -NULL -56 1969-12-31 16:00:08.868 -56.0 NULL 56 -56 0 -56.0 -0.0 56.0 -4454.968 0.0 56.0 0.0 -10.175 -56.0 0.4692857142857143 -56.0 0.0 -56 -NULL -15 1969-12-31 16:00:16.239 -15.0 NULL 15 -15 0 -15.0 -0.0 15.0 -1193.295 0.0 15.0 0.0 -10.175 -15.0 1.752 -15.0 0.0 -15 -NULL -26 1969-12-31 16:00:13.117 -26.0 NULL 26 -26 0 -26.0 -0.0 26.0 -2068.3779999999997 0.0 26.0 0.0 -10.175 -26.0 1.0107692307692309 -26.0 0.0 -26 -true -47 1969-12-31 16:00:05.786 -47.0 821UdmGbkEf4j 47 -47 0 -47.0 -0.0 47.0 -3738.991 0.0 47.0 0.0 -10.175 -47.0 0.5591489361702128 -47.0 0.0 -47 -NULL 45 1969-12-31 16:00:08.378 45.0 NULL -45 45 0 45.0 0.0 -45.0 3579.8849999999998 0.0 -45.0 0.0 -10.175 45.0 -0.5840000000000001 45.0 0.0 45 -NULL -13 1969-12-31 16:00:06.004 -13.0 NULL 13 -13 0 -13.0 -0.0 13.0 -1034.1889999999999 0.0 13.0 0.0 -10.175 -13.0 2.0215384615384617 -13.0 0.0 -13 -NULL 26 1969-12-31 16:00:10.506 26.0 NULL -26 26 0 26.0 0.0 -26.0 2068.3779999999997 0.0 -26.0 0.0 -10.175 26.0 -1.0107692307692309 26.0 0.0 26 -true -9 1969-12-31 16:00:10.652 -9.0 cvLH6Eat2yFsyy7p 9 -9 0 -9.0 -0.0 9.0 -715.977 0.0 9.0 0.0 -10.175 -9.0 2.92 -9.0 0.0 -9 -NULL -57 1969-12-31 16:00:12.626 -57.0 NULL 57 -57 0 -57.0 -0.0 57.0 -4534.521 0.0 57.0 0.0 -10.175 -57.0 0.4610526315789474 -57.0 0.0 -57 -NULL -21 1969-12-31 16:00:09.182 -21.0 NULL 21 -21 0 -21.0 -0.0 21.0 -1670.6129999999998 0.0 21.0 0.0 -10.175 -21.0 1.2514285714285716 -21.0 0.0 -21 -true 6 1969-12-31 16:00:09.107 6.0 821UdmGbkEf4j -6 6 0 6.0 0.0 -6.0 477.318 0.0 -6.0 0.0 -10.175 6.0 -4.38 6.0 0.0 6 -NULL 56 1969-12-31 16:00:10.767 56.0 NULL -56 56 0 56.0 0.0 -56.0 4454.968 0.0 -56.0 0.0 -10.175 56.0 -0.4692857142857143 56.0 0.0 56 -true 12 1969-12-31 16:00:13.383 12.0 821UdmGbkEf4j -12 12 0 12.0 0.0 -12.0 954.636 0.0 -12.0 0.0 -10.175 12.0 -2.19 12.0 0.0 12 -NULL 47 1969-12-31 16:00:15.105 47.0 NULL -47 47 0 47.0 0.0 -47.0 3738.991 0.0 -47.0 0.0 -10.175 47.0 -0.5591489361702128 47.0 0.0 47 -true 43 1969-12-31 16:00:13.539 43.0 821UdmGbkEf4j -43 43 0 43.0 0.0 -43.0 3420.779 0.0 -43.0 0.0 -10.175 43.0 -0.6111627906976744 43.0 0.0 43 -true -2 1969-12-31 16:00:13.347 -2.0 cvLH6Eat2yFsyy7p 2 -2 0 -2.0 -0.0 2.0 -159.106 0.0 2.0 0.0 -10.175 -2.0 13.14 -2.0 0.0 -2 -NULL -58 1969-12-31 16:00:14.066 -58.0 NULL 58 -58 0 -58.0 -0.0 58.0 -4614.074 0.0 58.0 0.0 -10.175 -58.0 0.4531034482758621 -58.0 0.0 -58 -NULL 32 1969-12-31 16:00:04.928 32.0 NULL -32 32 0 32.0 0.0 -32.0 2545.696 0.0 -32.0 0.0 -10.175 32.0 -0.82125 32.0 0.0 32 -NULL 39 1969-12-31 16:00:02.804 39.0 NULL -39 39 0 39.0 0.0 -39.0 3102.567 0.0 -39.0 0.0 -10.175 39.0 -0.6738461538461539 39.0 0.0 39 -NULL 57 1969-12-31 16:00:00.363 57.0 NULL -57 57 0 57.0 0.0 -57.0 4534.521 0.0 -57.0 0.0 -10.175 57.0 -0.4610526315789474 57.0 0.0 57 -NULL 16 1969-12-31 16:00:02.582 16.0 NULL -16 16 0 16.0 0.0 -16.0 1272.848 0.0 -16.0 0.0 -10.175 16.0 -1.6425 16.0 0.0 16 -NULL -29 1969-12-31 16:00:11.145 -29.0 NULL 29 -29 0 -29.0 -0.0 29.0 -2307.037 0.0 29.0 0.0 -10.175 -29.0 0.9062068965517242 -29.0 0.0 -29 -true -32 1969-12-31 16:00:07.118 -32.0 821UdmGbkEf4j 32 -32 0 -32.0 -0.0 32.0 -2545.696 0.0 32.0 0.0 -10.175 -32.0 0.82125 -32.0 0.0 -32 -NULL 19 1969-12-31 16:00:08.201 19.0 NULL -19 19 0 19.0 0.0 -19.0 1511.507 0.0 -19.0 0.0 -10.175 19.0 -1.3831578947368421 19.0 0.0 19 -NULL 36 1969-12-31 16:00:05.234 36.0 NULL -36 36 0 36.0 0.0 -36.0 2863.908 0.0 -36.0 0.0 -10.175 36.0 -0.73 36.0 0.0 36 -true 17 1969-12-31 16:00:00.225 17.0 821UdmGbkEf4j -17 17 0 17.0 0.0 -17.0 1352.4009999999998 0.0 -17.0 0.0 -10.175 17.0 -1.5458823529411765 17.0 0.0 17 -true -60 1969-12-31 16:00:16.165 -60.0 cvLH6Eat2yFsyy7p 60 -60 0 -60.0 -0.0 60.0 -4773.18 0.0 60.0 0.0 -10.175 -60.0 0.438 -60.0 0.0 -60 -true -23 1969-12-31 16:00:10.268 -23.0 1cGVWH7n1QU 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -true 15 1969-12-31 16:00:08.54 15.0 cvLH6Eat2yFsyy7p -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -NULL 45 1969-12-31 16:00:02.135 45.0 NULL -45 45 0 45.0 0.0 -45.0 3579.8849999999998 0.0 -45.0 0.0 -10.175 45.0 -0.5840000000000001 45.0 0.0 45 -true 43 1969-12-31 16:00:07.891 43.0 cvLH6Eat2yFsyy7p -43 43 0 43.0 0.0 -43.0 3420.779 0.0 -43.0 0.0 -10.175 43.0 -0.6111627906976744 43.0 0.0 43 -NULL -38 1969-12-31 16:00:14.127 -38.0 NULL 38 -38 0 -38.0 -0.0 38.0 -3023.014 0.0 38.0 0.0 -10.175 -38.0 0.6915789473684211 -38.0 0.0 -38 -NULL -16 1969-12-31 16:00:01.088 -16.0 NULL 16 -16 0 -16.0 -0.0 16.0 -1272.848 0.0 16.0 0.0 -10.175 -16.0 1.6425 -16.0 0.0 -16 -NULL -20 1969-12-31 16:00:07.942 -20.0 NULL 20 -20 0 -20.0 -0.0 20.0 -1591.06 0.0 20.0 0.0 -10.175 -20.0 1.314 -20.0 0.0 -20 -NULL 33 1969-12-31 16:00:00.318 33.0 NULL -33 33 0 33.0 0.0 -33.0 2625.249 0.0 -33.0 0.0 -10.175 33.0 -0.7963636363636364 33.0 0.0 33 -NULL -28 1969-12-31 16:00:04.149 -28.0 NULL 28 -28 0 -28.0 -0.0 28.0 -2227.484 0.0 28.0 0.0 -10.175 -28.0 0.9385714285714286 -28.0 0.0 -28 -NULL -10 1969-12-31 16:00:00.553 -10.0 NULL 10 -10 0 -10.0 -0.0 10.0 -795.53 0.0 10.0 0.0 -10.175 -10.0 2.628 -10.0 0.0 -10 -true -24 1969-12-31 16:00:06.756 -24.0 1cGVWH7n1QU 24 -24 0 -24.0 -0.0 24.0 -1909.272 0.0 24.0 0.0 -10.175 -24.0 1.095 -24.0 0.0 -24 -NULL 17 1969-12-31 16:00:15.082 17.0 NULL -17 17 0 17.0 0.0 -17.0 1352.4009999999998 0.0 -17.0 0.0 -10.175 17.0 -1.5458823529411765 17.0 0.0 17 -NULL -4 1969-12-31 16:00:13.237 -4.0 NULL 4 -4 0 -4.0 -0.0 4.0 -318.212 0.0 4.0 0.0 -10.175 -4.0 6.57 -4.0 0.0 -4 -NULL -12 1969-12-31 16:00:15.749 -12.0 NULL 12 -12 0 -12.0 -0.0 12.0 -954.636 0.0 12.0 0.0 -10.175 -12.0 2.19 -12.0 0.0 -12 -NULL -31 1969-12-31 16:00:13.552 -31.0 NULL 31 -31 0 -31.0 -0.0 31.0 -2466.143 0.0 31.0 0.0 -10.175 -31.0 0.847741935483871 -31.0 0.0 -31 -true -56 1969-12-31 16:00:14.038 -56.0 1cGVWH7n1QU 56 -56 0 -56.0 -0.0 56.0 -4454.968 0.0 56.0 0.0 -10.175 -56.0 0.4692857142857143 -56.0 0.0 -56 -NULL -7 1969-12-31 16:00:10.957 -7.0 NULL 7 -7 0 -7.0 -0.0 7.0 -556.871 0.0 7.0 0.0 -10.175 -7.0 3.7542857142857144 -7.0 0.0 -7 -true 52 1969-12-31 16:00:06.556 52.0 821UdmGbkEf4j -52 52 0 52.0 0.0 -52.0 4136.755999999999 0.0 -52.0 0.0 -10.175 52.0 -0.5053846153846154 52.0 0.0 52 -true -26 1969-12-31 16:00:05.641 -26.0 821UdmGbkEf4j 26 -26 0 -26.0 -0.0 26.0 -2068.3779999999997 0.0 26.0 0.0 -10.175 -26.0 1.0107692307692309 -26.0 0.0 -26 -NULL 35 1969-12-31 16:00:11.356 35.0 NULL -35 35 0 35.0 0.0 -35.0 2784.355 0.0 -35.0 0.0 -10.175 35.0 -0.7508571428571429 35.0 0.0 35 -true 7 1969-12-31 16:00:12.06 7.0 cvLH6Eat2yFsyy7p -7 7 0 7.0 0.0 -7.0 556.871 0.0 -7.0 0.0 -10.175 7.0 -3.7542857142857144 7.0 0.0 7 -NULL -5 1969-12-31 16:00:10.249 -5.0 NULL 5 -5 0 -5.0 -0.0 5.0 -397.765 0.0 5.0 0.0 -10.175 -5.0 5.256 -5.0 0.0 -5 -true 18 1969-12-31 16:00:09.747 18.0 1cGVWH7n1QU -18 18 0 18.0 0.0 -18.0 1431.954 0.0 -18.0 0.0 -10.175 18.0 -1.46 18.0 0.0 18 -true -63 1969-12-31 16:00:09.14 -63.0 821UdmGbkEf4j 63 -63 0 -63.0 -0.0 63.0 -5011.839 0.0 63.0 0.0 -10.175 -63.0 0.41714285714285715 -63.0 0.0 -63 -NULL -23 1969-12-31 16:00:12.317 -23.0 NULL 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -true 15 1969-12-31 16:00:06.182 15.0 1cGVWH7n1QU -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -true -15 1969-12-31 16:00:02.719 -15.0 821UdmGbkEf4j 15 -15 0 -15.0 -0.0 15.0 -1193.295 0.0 15.0 0.0 -10.175 -15.0 1.752 -15.0 0.0 -15 -true -38 1969-12-31 16:00:11.404 -38.0 1cGVWH7n1QU 38 -38 0 -38.0 -0.0 38.0 -3023.014 0.0 38.0 0.0 -10.175 -38.0 0.6915789473684211 -38.0 0.0 -38 -true -10 1969-12-31 16:00:08.086 -10.0 1cGVWH7n1QU 10 -10 0 -10.0 -0.0 10.0 -795.53 0.0 10.0 0.0 -10.175 -10.0 2.628 -10.0 0.0 -10 -NULL 31 1969-12-31 16:00:08.03 31.0 NULL -31 31 0 31.0 0.0 -31.0 2466.143 0.0 -31.0 0.0 -10.175 31.0 -0.847741935483871 31.0 0.0 31 -NULL -62 1969-12-31 16:00:14.965 -62.0 NULL 62 -62 0 -62.0 -0.0 62.0 -4932.286 0.0 62.0 0.0 -10.175 -62.0 0.4238709677419355 -62.0 0.0 -62 -NULL 40 1969-12-31 16:00:15.311 40.0 NULL -40 40 0 40.0 0.0 -40.0 3182.12 0.0 -40.0 0.0 -10.175 40.0 -0.657 40.0 0.0 40 -NULL 27 1969-12-31 16:00:03.727 27.0 NULL -27 27 0 27.0 0.0 -27.0 2147.931 0.0 -27.0 0.0 -10.175 27.0 -0.9733333333333334 27.0 0.0 27 -true -28 1969-12-31 16:00:00.74 -28.0 cvLH6Eat2yFsyy7p 28 -28 0 -28.0 -0.0 28.0 -2227.484 0.0 28.0 0.0 -10.175 -28.0 0.9385714285714286 -28.0 0.0 -28 -true -13 1969-12-31 16:00:03.377 -13.0 821UdmGbkEf4j 13 -13 0 -13.0 -0.0 13.0 -1034.1889999999999 0.0 13.0 0.0 -10.175 -13.0 2.0215384615384617 -13.0 0.0 -13 -true 28 1969-12-31 16:00:09.36 28.0 821UdmGbkEf4j -28 28 0 28.0 0.0 -28.0 2227.484 0.0 -28.0 0.0 -10.175 28.0 -0.9385714285714286 28.0 0.0 28 -true 51 1969-12-31 16:00:07.476 51.0 cvLH6Eat2yFsyy7p -51 51 0 51.0 0.0 -51.0 4057.203 0.0 -51.0 0.0 -10.175 51.0 -0.5152941176470588 51.0 0.0 51 -true 33 1969-12-31 16:00:15.717 33.0 821UdmGbkEf4j -33 33 0 33.0 0.0 -33.0 2625.249 0.0 -33.0 0.0 -10.175 33.0 -0.7963636363636364 33.0 0.0 33 -true -57 1969-12-31 16:00:13.365 -57.0 1cGVWH7n1QU 57 -57 0 -57.0 -0.0 57.0 -4534.521 0.0 57.0 0.0 -10.175 -57.0 0.4610526315789474 -57.0 0.0 -57 -true 13 1969-12-31 16:00:08.347 13.0 821UdmGbkEf4j -13 13 0 13.0 0.0 -13.0 1034.1889999999999 0.0 -13.0 0.0 -10.175 13.0 -2.0215384615384617 13.0 0.0 13 -true 9 1969-12-31 16:00:00.278 9.0 1cGVWH7n1QU -9 9 0 9.0 0.0 -9.0 715.977 0.0 -9.0 0.0 -10.175 9.0 -2.92 9.0 0.0 9 -NULL -63 1969-12-31 16:00:10.205 -63.0 NULL 63 -63 0 -63.0 -0.0 63.0 -5011.839 0.0 63.0 0.0 -10.175 -63.0 0.41714285714285715 -63.0 0.0 -63 -NULL -31 1969-12-31 16:00:13.868 -31.0 NULL 31 -31 0 -31.0 -0.0 31.0 -2466.143 0.0 31.0 0.0 -10.175 -31.0 0.847741935483871 -31.0 0.0 -31 -NULL -2 1969-12-31 16:00:06.498 -2.0 NULL 2 -2 0 -2.0 -0.0 2.0 -159.106 0.0 2.0 0.0 -10.175 -2.0 13.14 -2.0 0.0 -2 -NULL -19 1969-12-31 16:00:04.301 -19.0 NULL 19 -19 0 -19.0 -0.0 19.0 -1511.507 0.0 19.0 0.0 -10.175 -19.0 1.3831578947368421 -19.0 0.0 -19 -NULL 35 1969-12-31 16:00:10.259 35.0 NULL -35 35 0 35.0 0.0 -35.0 2784.355 0.0 -35.0 0.0 -10.175 35.0 -0.7508571428571429 35.0 0.0 35 -true 16 1969-12-31 16:00:11.214 16.0 821UdmGbkEf4j -16 16 0 16.0 0.0 -16.0 1272.848 0.0 -16.0 0.0 -10.175 16.0 -1.6425 16.0 0.0 16 -true -39 1969-12-31 16:00:12.799 -39.0 1cGVWH7n1QU 39 -39 0 -39.0 -0.0 39.0 -3102.567 0.0 39.0 0.0 -10.175 -39.0 0.6738461538461539 -39.0 0.0 -39 -NULL 15 1969-12-31 16:00:12.58 15.0 NULL -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -true -19 1969-12-31 16:00:10.174 -19.0 821UdmGbkEf4j 19 -19 0 -19.0 -0.0 19.0 -1511.507 0.0 19.0 0.0 -10.175 -19.0 1.3831578947368421 -19.0 0.0 -19 -NULL 30 1969-12-31 16:00:06.315 30.0 NULL -30 30 0 30.0 0.0 -30.0 2386.59 0.0 -30.0 0.0 -10.175 30.0 -0.876 30.0 0.0 30 -NULL 15 1969-12-31 16:00:05.63 15.0 NULL -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -NULL 28 1969-12-31 16:00:16.279 28.0 NULL -28 28 0 28.0 0.0 -28.0 2227.484 0.0 -28.0 0.0 -10.175 28.0 -0.9385714285714286 28.0 0.0 28 -true 6 1969-12-31 16:00:04.89 6.0 1cGVWH7n1QU -6 6 0 6.0 0.0 -6.0 477.318 0.0 -6.0 0.0 -10.175 6.0 -4.38 6.0 0.0 6 -NULL 58 1969-12-31 16:00:15.841 58.0 NULL -58 58 0 58.0 0.0 -58.0 4614.074 0.0 -58.0 0.0 -10.175 58.0 -0.4531034482758621 58.0 0.0 58 -NULL -63 1969-12-31 16:00:07.375 -63.0 NULL 63 -63 0 -63.0 -0.0 63.0 -5011.839 0.0 63.0 0.0 -10.175 -63.0 0.41714285714285715 -63.0 0.0 -63 -true 62 1969-12-31 16:00:09.442 62.0 821UdmGbkEf4j -62 62 0 62.0 0.0 -62.0 4932.286 0.0 -62.0 0.0 -10.175 62.0 -0.4238709677419355 62.0 0.0 62 -NULL 40 1969-12-31 16:00:12.353 40.0 NULL -40 40 0 40.0 0.0 -40.0 3182.12 0.0 -40.0 0.0 -10.175 40.0 -0.657 40.0 0.0 40 -NULL 16 1969-12-31 16:00:00.999 16.0 NULL -16 16 0 16.0 0.0 -16.0 1272.848 0.0 -16.0 0.0 -10.175 16.0 -1.6425 16.0 0.0 16 -NULL 17 1969-12-31 16:00:01.235 17.0 NULL -17 17 0 17.0 0.0 -17.0 1352.4009999999998 0.0 -17.0 0.0 -10.175 17.0 -1.5458823529411765 17.0 0.0 17 -NULL -31 1969-12-31 16:00:07.398 -31.0 NULL 31 -31 0 -31.0 -0.0 31.0 -2466.143 0.0 31.0 0.0 -10.175 -31.0 0.847741935483871 -31.0 0.0 -31 -true -36 1969-12-31 16:00:00.186 -36.0 cvLH6Eat2yFsyy7p 36 -36 0 -36.0 -0.0 36.0 -2863.908 0.0 36.0 0.0 -10.175 -36.0 0.73 -36.0 0.0 -36 -NULL 56 1969-12-31 16:00:11.845 56.0 NULL -56 56 0 56.0 0.0 -56.0 4454.968 0.0 -56.0 0.0 -10.175 56.0 -0.4692857142857143 56.0 0.0 56 -true -34 1969-12-31 16:00:03.97 -34.0 1cGVWH7n1QU 34 -34 0 -34.0 -0.0 34.0 -2704.8019999999997 0.0 34.0 0.0 -10.175 -34.0 0.7729411764705882 -34.0 0.0 -34 -true -54 1969-12-31 16:00:02.201 -54.0 cvLH6Eat2yFsyy7p 54 -54 0 -54.0 -0.0 54.0 -4295.862 0.0 54.0 0.0 -10.175 -54.0 0.4866666666666667 -54.0 0.0 -54 -true -46 1969-12-31 16:00:01.274 -46.0 cvLH6Eat2yFsyy7p 46 -46 0 -46.0 -0.0 46.0 -3659.438 0.0 46.0 0.0 -10.175 -46.0 0.571304347826087 -46.0 0.0 -46 -NULL 20 1969-12-31 16:00:05.147 20.0 NULL -20 20 0 20.0 0.0 -20.0 1591.06 0.0 -20.0 0.0 -10.175 20.0 -1.314 20.0 0.0 20 -NULL 9 1969-12-31 16:00:04.745 9.0 NULL -9 9 0 9.0 0.0 -9.0 715.977 0.0 -9.0 0.0 -10.175 9.0 -2.92 9.0 0.0 9 -NULL -43 1969-12-31 16:00:05.132 -43.0 NULL 43 -43 0 -43.0 -0.0 43.0 -3420.779 0.0 43.0 0.0 -10.175 -43.0 0.6111627906976744 -43.0 0.0 -43 -true -52 1969-12-31 16:00:06.732 -52.0 821UdmGbkEf4j 52 -52 0 -52.0 -0.0 52.0 -4136.755999999999 0.0 52.0 0.0 -10.175 -52.0 0.5053846153846154 -52.0 0.0 -52 -true -32 1969-12-31 16:00:09.353 -32.0 821UdmGbkEf4j 32 -32 0 -32.0 -0.0 32.0 -2545.696 0.0 32.0 0.0 -10.175 -32.0 0.82125 -32.0 0.0 -32 -NULL -46 1969-12-31 16:00:02.285 -46.0 NULL 46 -46 0 -46.0 -0.0 46.0 -3659.438 0.0 46.0 0.0 -10.175 -46.0 0.571304347826087 -46.0 0.0 -46 -NULL 57 1969-12-31 16:00:09.441 57.0 NULL -57 57 0 57.0 0.0 -57.0 4534.521 0.0 -57.0 0.0 -10.175 57.0 -0.4610526315789474 57.0 0.0 57 -NULL 49 1969-12-31 16:00:08.301 49.0 NULL -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 -true -51 1969-12-31 16:00:13.026 -51.0 cvLH6Eat2yFsyy7p 51 -51 0 -51.0 -0.0 51.0 -4057.203 0.0 51.0 0.0 -10.175 -51.0 0.5152941176470588 -51.0 0.0 -51 -true 50 1969-12-31 16:00:16.096 50.0 821UdmGbkEf4j -50 50 0 50.0 0.0 -50.0 3977.6499999999996 0.0 -50.0 0.0 -10.175 50.0 -0.5256000000000001 50.0 0.0 50 -true -49 1969-12-31 16:00:04.977 -49.0 cvLH6Eat2yFsyy7p 49 -49 0 -49.0 -0.0 49.0 -3898.0969999999998 0.0 49.0 0.0 -10.175 -49.0 0.5363265306122449 -49.0 0.0 -49 -NULL -16 1969-12-31 16:00:12.574 -16.0 NULL 16 -16 0 -16.0 -0.0 16.0 -1272.848 0.0 16.0 0.0 -10.175 -16.0 1.6425 -16.0 0.0 -16 -true 50 1969-12-31 16:00:15.443 50.0 cvLH6Eat2yFsyy7p -50 50 0 50.0 0.0 -50.0 3977.6499999999996 0.0 -50.0 0.0 -10.175 50.0 -0.5256000000000001 50.0 0.0 50 -true -63 1969-12-31 16:00:13.752 -63.0 cvLH6Eat2yFsyy7p 63 -63 0 -63.0 -0.0 63.0 -5011.839 0.0 63.0 0.0 -10.175 -63.0 0.41714285714285715 -63.0 0.0 -63 -true -44 1969-12-31 16:00:12.936 -44.0 821UdmGbkEf4j 44 -44 0 -44.0 -0.0 44.0 -3500.332 0.0 44.0 0.0 -10.175 -44.0 0.5972727272727273 -44.0 0.0 -44 -true -1 1969-12-31 16:00:00.107 -1.0 1cGVWH7n1QU 1 -1 0 -1.0 -0.0 1.0 -79.553 0.0 1.0 0.0 -10.175 -1.0 26.28 -1.0 0.0 -1 -NULL 15 1969-12-31 16:00:05.784 15.0 NULL -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -true -37 1969-12-31 16:00:05.792 -37.0 cvLH6Eat2yFsyy7p 37 -37 0 -37.0 -0.0 37.0 -2943.461 0.0 37.0 0.0 -10.175 -37.0 0.7102702702702703 -37.0 0.0 -37 -NULL 16 1969-12-31 16:00:15.143 16.0 NULL -16 16 0 16.0 0.0 -16.0 1272.848 0.0 -16.0 0.0 -10.175 16.0 -1.6425 16.0 0.0 16 -true -41 1969-12-31 16:00:16.139 -41.0 1cGVWH7n1QU 41 -41 0 -41.0 -0.0 41.0 -3261.673 0.0 41.0 0.0 -10.175 -41.0 0.6409756097560976 -41.0 0.0 -41 -true -34 1969-12-31 16:00:00.963 -34.0 1cGVWH7n1QU 34 -34 0 -34.0 -0.0 34.0 -2704.8019999999997 0.0 34.0 0.0 -10.175 -34.0 0.7729411764705882 -34.0 0.0 -34 -NULL -7 1969-12-31 16:00:02.436 -7.0 NULL 7 -7 0 -7.0 -0.0 7.0 -556.871 0.0 7.0 0.0 -10.175 -7.0 3.7542857142857144 -7.0 0.0 -7 -true -30 1969-12-31 16:00:07.062 -30.0 1cGVWH7n1QU 30 -30 0 -30.0 -0.0 30.0 -2386.59 0.0 30.0 0.0 -10.175 -30.0 0.876 -30.0 0.0 -30 -NULL -6 1969-12-31 16:00:07.423 -6.0 NULL 6 -6 0 -6.0 -0.0 6.0 -477.318 0.0 6.0 0.0 -10.175 -6.0 4.38 -6.0 0.0 -6 -NULL -62 1969-12-31 16:00:10.069 -62.0 NULL 62 -62 0 -62.0 -0.0 62.0 -4932.286 0.0 62.0 0.0 -10.175 -62.0 0.4238709677419355 -62.0 0.0 -62 -NULL -23 1969-12-31 16:00:11.402 -23.0 NULL 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -NULL -47 1969-12-31 16:00:00.555 -47.0 NULL 47 -47 0 -47.0 -0.0 47.0 -3738.991 0.0 47.0 0.0 -10.175 -47.0 0.5591489361702128 -47.0 0.0 -47 -NULL 47 1969-12-31 16:00:01.694 47.0 NULL -47 47 0 47.0 0.0 -47.0 3738.991 0.0 -47.0 0.0 -10.175 47.0 -0.5591489361702128 47.0 0.0 47 -NULL 5 1969-12-31 16:00:08.427 5.0 NULL -5 5 0 5.0 0.0 -5.0 397.765 0.0 -5.0 0.0 -10.175 5.0 -5.256 5.0 0.0 5 -NULL -9 1969-12-31 16:00:10.452 -9.0 NULL 9 -9 0 -9.0 -0.0 9.0 -715.977 0.0 9.0 0.0 -10.175 -9.0 2.92 -9.0 0.0 -9 -true -16 1969-12-31 16:00:08.035 -16.0 cvLH6Eat2yFsyy7p 16 -16 0 -16.0 -0.0 16.0 -1272.848 0.0 16.0 0.0 -10.175 -16.0 1.6425 -16.0 0.0 -16 -true 15 1969-12-31 16:00:00.465 15.0 cvLH6Eat2yFsyy7p -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -true 13 1969-12-31 16:00:04.711 13.0 821UdmGbkEf4j -13 13 0 13.0 0.0 -13.0 1034.1889999999999 0.0 -13.0 0.0 -10.175 13.0 -2.0215384615384617 13.0 0.0 13 -NULL -36 1969-12-31 16:00:09.804 -36.0 NULL 36 -36 0 -36.0 -0.0 36.0 -2863.908 0.0 36.0 0.0 -10.175 -36.0 0.73 -36.0 0.0 -36 -true 58 1969-12-31 16:00:02.777 58.0 1cGVWH7n1QU -58 58 0 58.0 0.0 -58.0 4614.074 0.0 -58.0 0.0 -10.175 58.0 -0.4531034482758621 58.0 0.0 58 -NULL 62 1969-12-31 16:00:04.967 62.0 NULL -62 62 0 62.0 0.0 -62.0 4932.286 0.0 -62.0 0.0 -10.175 62.0 -0.4238709677419355 62.0 0.0 62 -true -34 1969-12-31 16:00:01.278 -34.0 1cGVWH7n1QU 34 -34 0 -34.0 -0.0 34.0 -2704.8019999999997 0.0 34.0 0.0 -10.175 -34.0 0.7729411764705882 -34.0 0.0 -34 -true -56 1969-12-31 16:00:07.364 -56.0 cvLH6Eat2yFsyy7p 56 -56 0 -56.0 -0.0 56.0 -4454.968 0.0 56.0 0.0 -10.175 -56.0 0.4692857142857143 -56.0 0.0 -56 -NULL 55 1969-12-31 16:00:02.215 55.0 NULL -55 55 0 55.0 0.0 -55.0 4375.415 0.0 -55.0 0.0 -10.175 55.0 -0.47781818181818186 55.0 0.0 55 -true 25 1969-12-31 16:00:02.319 25.0 821UdmGbkEf4j -25 25 0 25.0 0.0 -25.0 1988.8249999999998 0.0 -25.0 0.0 -10.175 25.0 -1.0512000000000001 25.0 0.0 25 -NULL 34 1969-12-31 16:00:07.045 34.0 NULL -34 34 0 34.0 0.0 -34.0 2704.8019999999997 0.0 -34.0 0.0 -10.175 34.0 -0.7729411764705882 34.0 0.0 34 -NULL -10 1969-12-31 16:00:13.743 -10.0 NULL 10 -10 0 -10.0 -0.0 10.0 -795.53 0.0 10.0 0.0 -10.175 -10.0 2.628 -10.0 0.0 -10 -NULL 25 1969-12-31 16:00:11.749 25.0 NULL -25 25 0 25.0 0.0 -25.0 1988.8249999999998 0.0 -25.0 0.0 -10.175 25.0 -1.0512000000000001 25.0 0.0 25 -NULL -64 1969-12-31 16:00:00.29 -64.0 NULL 64 -64 0 -64.0 -0.0 64.0 -5091.392 0.0 64.0 0.0 -10.175 -64.0 0.410625 -64.0 0.0 -64 -NULL 13 1969-12-31 16:00:00.025 13.0 NULL -13 13 0 13.0 0.0 -13.0 1034.1889999999999 0.0 -13.0 0.0 -10.175 13.0 -2.0215384615384617 13.0 0.0 13 -true -37 1969-12-31 16:00:09.597 -37.0 cvLH6Eat2yFsyy7p 37 -37 0 -37.0 -0.0 37.0 -2943.461 0.0 37.0 0.0 -10.175 -37.0 0.7102702702702703 -37.0 0.0 -37 -true -6 1969-12-31 16:00:05.07 -6.0 cvLH6Eat2yFsyy7p 6 -6 0 -6.0 -0.0 6.0 -477.318 0.0 6.0 0.0 -10.175 -6.0 4.38 -6.0 0.0 -6 -true -34 1969-12-31 16:00:10.346 -34.0 cvLH6Eat2yFsyy7p 34 -34 0 -34.0 -0.0 34.0 -2704.8019999999997 0.0 34.0 0.0 -10.175 -34.0 0.7729411764705882 -34.0 0.0 -34 -true 14 1969-12-31 16:00:12.781 14.0 cvLH6Eat2yFsyy7p -14 14 0 14.0 0.0 -14.0 1113.742 0.0 -14.0 0.0 -10.175 14.0 -1.8771428571428572 14.0 0.0 14 -NULL -23 1969-12-31 16:00:16.07 -23.0 NULL 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -NULL 27 1969-12-31 16:00:06.511 27.0 NULL -27 27 0 27.0 0.0 -27.0 2147.931 0.0 -27.0 0.0 -10.175 27.0 -0.9733333333333334 27.0 0.0 27 -NULL 29 1969-12-31 16:00:02.432 29.0 NULL -29 29 0 29.0 0.0 -29.0 2307.037 0.0 -29.0 0.0 -10.175 29.0 -0.9062068965517242 29.0 0.0 29 -true 11 1969-12-31 16:00:09.532 11.0 1cGVWH7n1QU -11 11 0 11.0 0.0 -11.0 875.083 0.0 -11.0 0.0 -10.175 11.0 -2.389090909090909 11.0 0.0 11 -NULL 15 1969-12-31 16:00:03.256 15.0 NULL -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -NULL 45 1969-12-31 16:00:04.572 45.0 NULL -45 45 0 45.0 0.0 -45.0 3579.8849999999998 0.0 -45.0 0.0 -10.175 45.0 -0.5840000000000001 45.0 0.0 45 -true -31 1969-12-31 16:00:12.081 -31.0 cvLH6Eat2yFsyy7p 31 -31 0 -31.0 -0.0 31.0 -2466.143 0.0 31.0 0.0 -10.175 -31.0 0.847741935483871 -31.0 0.0 -31 -NULL -20 1969-12-31 16:00:15.161 -20.0 NULL 20 -20 0 -20.0 -0.0 20.0 -1591.06 0.0 20.0 0.0 -10.175 -20.0 1.314 -20.0 0.0 -20 -NULL 26 1969-12-31 16:00:00.923 26.0 NULL -26 26 0 26.0 0.0 -26.0 2068.3779999999997 0.0 -26.0 0.0 -10.175 26.0 -1.0107692307692309 26.0 0.0 26 -true 60 1969-12-31 16:00:05.986 60.0 1cGVWH7n1QU -60 60 0 60.0 0.0 -60.0 4773.18 0.0 -60.0 0.0 -10.175 60.0 -0.438 60.0 0.0 60 -true 25 1969-12-31 16:00:10.528 25.0 cvLH6Eat2yFsyy7p -25 25 0 25.0 0.0 -25.0 1988.8249999999998 0.0 -25.0 0.0 -10.175 25.0 -1.0512000000000001 25.0 0.0 25 -true -30 1969-12-31 16:00:14.072 -30.0 cvLH6Eat2yFsyy7p 30 -30 0 -30.0 -0.0 30.0 -2386.59 0.0 30.0 0.0 -10.175 -30.0 0.876 -30.0 0.0 -30 -true 47 1969-12-31 16:00:00.513 47.0 1cGVWH7n1QU -47 47 0 47.0 0.0 -47.0 3738.991 0.0 -47.0 0.0 -10.175 47.0 -0.5591489361702128 47.0 0.0 47 -NULL 61 1969-12-31 16:00:05.697 61.0 NULL -61 61 0 61.0 0.0 -61.0 4852.733 0.0 -61.0 0.0 -10.175 61.0 -0.4308196721311476 61.0 0.0 61 -true 59 1969-12-31 16:00:01.736 59.0 cvLH6Eat2yFsyy7p -59 59 0 59.0 0.0 -59.0 4693.6269999999995 0.0 -59.0 0.0 -10.175 59.0 -0.44542372881355935 59.0 0.0 59 -true 26 1969-12-31 16:00:16.191 26.0 821UdmGbkEf4j -26 26 0 26.0 0.0 -26.0 2068.3779999999997 0.0 -26.0 0.0 -10.175 26.0 -1.0107692307692309 26.0 0.0 26 -true -1 1969-12-31 16:00:01.502 -1.0 cvLH6Eat2yFsyy7p 1 -1 0 -1.0 -0.0 1.0 -79.553 0.0 1.0 0.0 -10.175 -1.0 26.28 -1.0 0.0 -1 -NULL -58 1969-12-31 16:00:06.255 -58.0 NULL 58 -58 0 -58.0 -0.0 58.0 -4614.074 0.0 58.0 0.0 -10.175 -58.0 0.4531034482758621 -58.0 0.0 -58 -true -64 1969-12-31 16:00:01.305 -64.0 cvLH6Eat2yFsyy7p 64 -64 0 -64.0 -0.0 64.0 -5091.392 0.0 64.0 0.0 -10.175 -64.0 0.410625 -64.0 0.0 -64 -NULL -17 1969-12-31 16:00:13.549 -17.0 NULL 17 -17 0 -17.0 -0.0 17.0 -1352.4009999999998 0.0 17.0 0.0 -10.175 -17.0 1.5458823529411765 -17.0 0.0 -17 -NULL -45 1969-12-31 16:00:05.68 -45.0 NULL 45 -45 0 -45.0 -0.0 45.0 -3579.8849999999998 0.0 45.0 0.0 -10.175 -45.0 0.5840000000000001 -45.0 0.0 -45 -true -1 1969-12-31 16:00:13.333 -1.0 1cGVWH7n1QU 1 -1 0 -1.0 -0.0 1.0 -79.553 0.0 1.0 0.0 -10.175 -1.0 26.28 -1.0 0.0 -1 -true -10 1969-12-31 16:00:00.619 -10.0 cvLH6Eat2yFsyy7p 10 -10 0 -10.0 -0.0 10.0 -795.53 0.0 10.0 0.0 -10.175 -10.0 2.628 -10.0 0.0 -10 -true 1 1969-12-31 16:00:09.74 1.0 821UdmGbkEf4j -1 1 0 1.0 0.0 -1.0 79.553 0.0 -1.0 0.0 -10.175 1.0 -26.28 1.0 0.0 1 -true 62 1969-12-31 16:00:09.659 62.0 821UdmGbkEf4j -62 62 0 62.0 0.0 -62.0 4932.286 0.0 -62.0 0.0 -10.175 62.0 -0.4238709677419355 62.0 0.0 62 -true -49 1969-12-31 16:00:10.097 -49.0 cvLH6Eat2yFsyy7p 49 -49 0 -49.0 -0.0 49.0 -3898.0969999999998 0.0 49.0 0.0 -10.175 -49.0 0.5363265306122449 -49.0 0.0 -49 -NULL -44 1969-12-31 16:00:10.898 -44.0 NULL 44 -44 0 -44.0 -0.0 44.0 -3500.332 0.0 44.0 0.0 -10.175 -44.0 0.5972727272727273 -44.0 0.0 -44 -true 59 1969-12-31 16:00:02.157 59.0 821UdmGbkEf4j -59 59 0 59.0 0.0 -59.0 4693.6269999999995 0.0 -59.0 0.0 -10.175 59.0 -0.44542372881355935 59.0 0.0 59 -true -55 1969-12-31 16:00:03.885 -55.0 821UdmGbkEf4j 55 -55 0 -55.0 -0.0 55.0 -4375.415 0.0 55.0 0.0 -10.175 -55.0 0.47781818181818186 -55.0 0.0 -55 -NULL -24 1969-12-31 16:00:06.866 -24.0 NULL 24 -24 0 -24.0 -0.0 24.0 -1909.272 0.0 24.0 0.0 -10.175 -24.0 1.095 -24.0 0.0 -24 -NULL -33 1969-12-31 16:00:11.95 -33.0 NULL 33 -33 0 -33.0 -0.0 33.0 -2625.249 0.0 33.0 0.0 -10.175 -33.0 0.7963636363636364 -33.0 0.0 -33 -true -25 1969-12-31 16:00:08.375 -25.0 cvLH6Eat2yFsyy7p 25 -25 0 -25.0 -0.0 25.0 -1988.8249999999998 0.0 25.0 0.0 -10.175 -25.0 1.0512000000000001 -25.0 0.0 -25 -true -39 1969-12-31 16:00:04.52 -39.0 1cGVWH7n1QU 39 -39 0 -39.0 -0.0 39.0 -3102.567 0.0 39.0 0.0 -10.175 -39.0 0.6738461538461539 -39.0 0.0 -39 -NULL -50 1969-12-31 16:00:15.764 -50.0 NULL 50 -50 0 -50.0 -0.0 50.0 -3977.6499999999996 0.0 50.0 0.0 -10.175 -50.0 0.5256000000000001 -50.0 0.0 -50 -true 33 1969-12-31 16:00:06.848 33.0 cvLH6Eat2yFsyy7p -33 33 0 33.0 0.0 -33.0 2625.249 0.0 -33.0 0.0 -10.175 33.0 -0.7963636363636364 33.0 0.0 33 -true 4 1969-12-31 16:00:15.754 4.0 821UdmGbkEf4j -4 4 0 4.0 0.0 -4.0 318.212 0.0 -4.0 0.0 -10.175 4.0 -6.57 4.0 0.0 4 -NULL -16 1969-12-31 16:00:16.127 -16.0 NULL 16 -16 0 -16.0 -0.0 16.0 -1272.848 0.0 16.0 0.0 -10.175 -16.0 1.6425 -16.0 0.0 -16 -NULL -48 1969-12-31 16:00:15.207 -48.0 NULL 48 -48 0 -48.0 -0.0 48.0 -3818.544 0.0 48.0 0.0 -10.175 -48.0 0.5475 -48.0 0.0 -48 -NULL 18 1969-12-31 16:00:13.689 18.0 NULL -18 18 0 18.0 0.0 -18.0 1431.954 0.0 -18.0 0.0 -10.175 18.0 -1.46 18.0 0.0 18 -NULL -37 1969-12-31 16:00:09.111 -37.0 NULL 37 -37 0 -37.0 -0.0 37.0 -2943.461 0.0 37.0 0.0 -10.175 -37.0 0.7102702702702703 -37.0 0.0 -37 -NULL -23 1969-12-31 16:00:11.284 -26.28 NULL 23 -23 0 -26.280000686645508 -0.0 26.280000686645508 -2090.65289462471 0.0 26.280000686645508 0.0 -10.175 -26.280000686645508 0.9999999738719373 -26.28 0.0 -23 -true 27 1969-12-31 16:00:00.957 27.0 1cGVWH7n1QU -27 27 0 27.0 0.0 -27.0 2147.931 0.0 -27.0 0.0 -10.175 27.0 -0.9733333333333334 27.0 0.0 27 -NULL -31 1969-12-31 16:00:11.841 -31.0 NULL 31 -31 0 -31.0 -0.0 31.0 -2466.143 0.0 31.0 0.0 -10.175 -31.0 0.847741935483871 -31.0 0.0 -31 -NULL 25 1969-12-31 16:00:06.38 25.0 NULL -25 25 0 25.0 0.0 -25.0 1988.8249999999998 0.0 -25.0 0.0 -10.175 25.0 -1.0512000000000001 25.0 0.0 25 -true -11 1969-12-31 16:00:11.149 -11.0 1cGVWH7n1QU 11 -11 0 -11.0 -0.0 11.0 -875.083 0.0 11.0 0.0 -10.175 -11.0 2.389090909090909 -11.0 0.0 -11 -true 21 1969-12-31 16:00:00.651 21.0 1cGVWH7n1QU -21 21 0 21.0 0.0 -21.0 1670.6129999999998 0.0 -21.0 0.0 -10.175 21.0 -1.2514285714285716 21.0 0.0 21 -true -26 1969-12-31 16:00:03.52 -26.0 cvLH6Eat2yFsyy7p 26 -26 0 -26.0 -0.0 26.0 -2068.3779999999997 0.0 26.0 0.0 -10.175 -26.0 1.0107692307692309 -26.0 0.0 -26 -NULL -38 1969-12-31 16:00:14.548 -38.0 NULL 38 -38 0 -38.0 -0.0 38.0 -3023.014 0.0 38.0 0.0 -10.175 -38.0 0.6915789473684211 -38.0 0.0 -38 -true 13 1969-12-31 16:00:12.554 13.0 cvLH6Eat2yFsyy7p -13 13 0 13.0 0.0 -13.0 1034.1889999999999 0.0 -13.0 0.0 -10.175 13.0 -2.0215384615384617 13.0 0.0 13 -true 52 1969-12-31 16:00:01.196 52.0 cvLH6Eat2yFsyy7p -52 52 0 52.0 0.0 -52.0 4136.755999999999 0.0 -52.0 0.0 -10.175 52.0 -0.5053846153846154 52.0 0.0 52 -NULL 40 1969-12-31 16:00:06.827 40.0 NULL -40 40 0 40.0 0.0 -40.0 3182.12 0.0 -40.0 0.0 -10.175 40.0 -0.657 40.0 0.0 40 -NULL -39 1969-12-31 16:00:08.449 -39.0 NULL 39 -39 0 -39.0 -0.0 39.0 -3102.567 0.0 39.0 0.0 -10.175 -39.0 0.6738461538461539 -39.0 0.0 -39 -NULL 17 1969-12-31 16:00:07.243 17.0 NULL -17 17 0 17.0 0.0 -17.0 1352.4009999999998 0.0 -17.0 0.0 -10.175 17.0 -1.5458823529411765 17.0 0.0 17 -NULL 40 1969-12-31 16:00:14.977 40.0 NULL -40 40 0 40.0 0.0 -40.0 3182.12 0.0 -40.0 0.0 -10.175 40.0 -0.657 40.0 0.0 40 -true -40 1969-12-31 16:00:07.846 -40.0 1cGVWH7n1QU 40 -40 0 -40.0 -0.0 40.0 -3182.12 0.0 40.0 0.0 -10.175 -40.0 0.657 -40.0 0.0 -40 -true -22 1969-12-31 16:00:14.318 -22.0 1cGVWH7n1QU 22 -22 0 -22.0 -0.0 22.0 -1750.166 0.0 22.0 0.0 -10.175 -22.0 1.1945454545454546 -22.0 0.0 -22 -NULL -52 1969-12-31 16:00:01.289 -52.0 NULL 52 -52 0 -52.0 -0.0 52.0 -4136.755999999999 0.0 52.0 0.0 -10.175 -52.0 0.5053846153846154 -52.0 0.0 -52 -true -49 1969-12-31 16:00:00.071 -49.0 cvLH6Eat2yFsyy7p 49 -49 0 -49.0 -0.0 49.0 -3898.0969999999998 0.0 49.0 0.0 -10.175 -49.0 0.5363265306122449 -49.0 0.0 -49 -NULL 18 1969-12-31 16:00:12.437 18.0 NULL -18 18 0 18.0 0.0 -18.0 1431.954 0.0 -18.0 0.0 -10.175 18.0 -1.46 18.0 0.0 18 -true -27 1969-12-31 16:00:04.256 -27.0 821UdmGbkEf4j 27 -27 0 -27.0 -0.0 27.0 -2147.931 0.0 27.0 0.0 -10.175 -27.0 0.9733333333333334 -27.0 0.0 -27 -NULL 47 1969-12-31 16:00:08.18 47.0 NULL -47 47 0 47.0 0.0 -47.0 3738.991 0.0 -47.0 0.0 -10.175 47.0 -0.5591489361702128 47.0 0.0 47 -NULL 19 1969-12-31 16:00:09.001 19.0 NULL -19 19 0 19.0 0.0 -19.0 1511.507 0.0 -19.0 0.0 -10.175 19.0 -1.3831578947368421 19.0 0.0 19 -NULL -47 1969-12-31 16:00:02.508 -47.0 NULL 47 -47 0 -47.0 -0.0 47.0 -3738.991 0.0 47.0 0.0 -10.175 -47.0 0.5591489361702128 -47.0 0.0 -47 -NULL 41 1969-12-31 16:00:01.424 41.0 NULL -41 41 0 41.0 0.0 -41.0 3261.673 0.0 -41.0 0.0 -10.175 41.0 -0.6409756097560976 41.0 0.0 41 -true 34 1969-12-31 16:00:13.221 34.0 821UdmGbkEf4j -34 34 0 34.0 0.0 -34.0 2704.8019999999997 0.0 -34.0 0.0 -10.175 34.0 -0.7729411764705882 34.0 0.0 34 -NULL -1 1969-12-31 16:00:06.358 -1.0 NULL 1 -1 0 -1.0 -0.0 1.0 -79.553 0.0 1.0 0.0 -10.175 -1.0 26.28 -1.0 0.0 -1 -true -14 1969-12-31 16:00:15.625 -14.0 cvLH6Eat2yFsyy7p 14 -14 0 -14.0 -0.0 14.0 -1113.742 0.0 14.0 0.0 -10.175 -14.0 1.8771428571428572 -14.0 0.0 -14 -true -53 1969-12-31 16:00:11.977 -53.0 821UdmGbkEf4j 53 -53 0 -53.0 -0.0 53.0 -4216.309 0.0 53.0 0.0 -10.175 -53.0 0.4958490566037736 -53.0 0.0 -53 -true 13 1969-12-31 16:00:10.815 13.0 1cGVWH7n1QU -13 13 0 13.0 0.0 -13.0 1034.1889999999999 0.0 -13.0 0.0 -10.175 13.0 -2.0215384615384617 13.0 0.0 13 -true -58 1969-12-31 16:00:13.209 -58.0 cvLH6Eat2yFsyy7p 58 -58 0 -58.0 -0.0 58.0 -4614.074 0.0 58.0 0.0 -10.175 -58.0 0.4531034482758621 -58.0 0.0 -58 -NULL 3 1969-12-31 16:00:00.074 3.0 NULL -3 3 0 3.0 0.0 -3.0 238.659 0.0 -3.0 0.0 -10.175 3.0 -8.76 3.0 0.0 3 -NULL -1 1969-12-31 16:00:04.836 -1.0 NULL 1 -1 0 -1.0 -0.0 1.0 -79.553 0.0 1.0 0.0 -10.175 -1.0 26.28 -1.0 0.0 -1 -NULL 60 1969-12-31 16:00:11.003 60.0 NULL -60 60 0 60.0 0.0 -60.0 4773.18 0.0 -60.0 0.0 -10.175 60.0 -0.438 60.0 0.0 60 -NULL 28 1969-12-31 16:00:05.451 28.0 NULL -28 28 0 28.0 0.0 -28.0 2227.484 0.0 -28.0 0.0 -10.175 28.0 -0.9385714285714286 28.0 0.0 28 -NULL -27 1969-12-31 16:00:14.065 -27.0 NULL 27 -27 0 -27.0 -0.0 27.0 -2147.931 0.0 27.0 0.0 -10.175 -27.0 0.9733333333333334 -27.0 0.0 -27 -NULL 15 1969-12-31 16:00:07.607 15.0 NULL -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -NULL 26 1969-12-31 16:00:10.285 26.0 NULL -26 26 0 26.0 0.0 -26.0 2068.3779999999997 0.0 -26.0 0.0 -10.175 26.0 -1.0107692307692309 26.0 0.0 26 -NULL -31 1969-12-31 16:00:01.702 -31.0 NULL 31 -31 0 -31.0 -0.0 31.0 -2466.143 0.0 31.0 0.0 -10.175 -31.0 0.847741935483871 -31.0 0.0 -31 -true -59 1969-12-31 16:00:12.227 -59.0 cvLH6Eat2yFsyy7p 59 -59 0 -59.0 -0.0 59.0 -4693.6269999999995 0.0 59.0 0.0 -10.175 -59.0 0.44542372881355935 -59.0 0.0 -59 -true 26 1969-12-31 16:00:02.396 26.0 821UdmGbkEf4j -26 26 0 26.0 0.0 -26.0 2068.3779999999997 0.0 -26.0 0.0 -10.175 26.0 -1.0107692307692309 26.0 0.0 26 -NULL 59 1969-12-31 16:00:09.343 59.0 NULL -59 59 0 59.0 0.0 -59.0 4693.6269999999995 0.0 -59.0 0.0 -10.175 59.0 -0.44542372881355935 59.0 0.0 59 -NULL -5 1969-12-31 16:00:12.18 -5.0 NULL 5 -5 0 -5.0 -0.0 5.0 -397.765 0.0 5.0 0.0 -10.175 -5.0 5.256 -5.0 0.0 -5 -true -12 1969-12-31 16:00:01.374 -12.0 821UdmGbkEf4j 12 -12 0 -12.0 -0.0 12.0 -954.636 0.0 12.0 0.0 -10.175 -12.0 2.19 -12.0 0.0 -12 -true -8 1969-12-31 16:00:08.365 -8.0 1cGVWH7n1QU 8 -8 0 -8.0 -0.0 8.0 -636.424 0.0 8.0 0.0 -10.175 -8.0 3.285 -8.0 0.0 -8 -NULL -50 1969-12-31 16:00:11.284 -50.0 NULL 50 -50 0 -50.0 -0.0 50.0 -3977.6499999999996 0.0 50.0 0.0 -10.175 -50.0 0.5256000000000001 -50.0 0.0 -50 -true -16 1969-12-31 16:00:01.416 -16.0 1cGVWH7n1QU 16 -16 0 -16.0 -0.0 16.0 -1272.848 0.0 16.0 0.0 -10.175 -16.0 1.6425 -16.0 0.0 -16 -NULL 0 1969-12-31 16:00:00.183 0.0 NULL 0 0 0 0.0 0.0 -0.0 0.0 0.0 -0.0 0.0 -10.175 0.0 NULL 0.0 NULL 0 -NULL 34 1969-12-31 16:00:09.65 34.0 NULL -34 34 0 34.0 0.0 -34.0 2704.8019999999997 0.0 -34.0 0.0 -10.175 34.0 -0.7729411764705882 34.0 0.0 34 -true -6 1969-12-31 16:00:07.372 -6.0 cvLH6Eat2yFsyy7p 6 -6 0 -6.0 -0.0 6.0 -477.318 0.0 6.0 0.0 -10.175 -6.0 4.38 -6.0 0.0 -6 -NULL -22 1969-12-31 16:00:12.901 -22.0 NULL 22 -22 0 -22.0 -0.0 22.0 -1750.166 0.0 22.0 0.0 -10.175 -22.0 1.1945454545454546 -22.0 0.0 -22 -true 62 1969-12-31 16:00:10.845 62.0 821UdmGbkEf4j -62 62 0 62.0 0.0 -62.0 4932.286 0.0 -62.0 0.0 -10.175 62.0 -0.4238709677419355 62.0 0.0 62 -true -5 1969-12-31 16:00:15.466 -5.0 1cGVWH7n1QU 5 -5 0 -5.0 -0.0 5.0 -397.765 0.0 5.0 0.0 -10.175 -5.0 5.256 -5.0 0.0 -5 -NULL 36 1969-12-31 16:00:06.494 36.0 NULL -36 36 0 36.0 0.0 -36.0 2863.908 0.0 -36.0 0.0 -10.175 36.0 -0.73 36.0 0.0 36 -NULL -29 1969-12-31 16:00:03.506 -29.0 NULL 29 -29 0 -29.0 -0.0 29.0 -2307.037 0.0 29.0 0.0 -10.175 -29.0 0.9062068965517242 -29.0 0.0 -29 -true -48 1969-12-31 16:00:06.725 -48.0 cvLH6Eat2yFsyy7p 48 -48 0 -48.0 -0.0 48.0 -3818.544 0.0 48.0 0.0 -10.175 -48.0 0.5475 -48.0 0.0 -48 -true -63 1969-12-31 16:00:05.654 -63.0 821UdmGbkEf4j 63 -63 0 -63.0 -0.0 63.0 -5011.839 0.0 63.0 0.0 -10.175 -63.0 0.41714285714285715 -63.0 0.0 -63 -true 6 1969-12-31 16:00:04.783 6.0 cvLH6Eat2yFsyy7p -6 6 0 6.0 0.0 -6.0 477.318 0.0 -6.0 0.0 -10.175 6.0 -4.38 6.0 0.0 6 -NULL -20 1969-12-31 16:00:11.847 -20.0 NULL 20 -20 0 -20.0 -0.0 20.0 -1591.06 0.0 20.0 0.0 -10.175 -20.0 1.314 -20.0 0.0 -20 -true 39 1969-12-31 16:00:00.102 39.0 cvLH6Eat2yFsyy7p -39 39 0 39.0 0.0 -39.0 3102.567 0.0 -39.0 0.0 -10.175 39.0 -0.6738461538461539 39.0 0.0 39 -NULL -61 1969-12-31 16:00:00.142 -61.0 NULL 61 -61 0 -61.0 -0.0 61.0 -4852.733 0.0 61.0 0.0 -10.175 -61.0 0.4308196721311476 -61.0 0.0 -61 -true 3 1969-12-31 16:00:03.321 3.0 cvLH6Eat2yFsyy7p -3 3 0 3.0 0.0 -3.0 238.659 0.0 -3.0 0.0 -10.175 3.0 -8.76 3.0 0.0 3 -true -13 1969-12-31 16:00:15.294 -13.0 1cGVWH7n1QU 13 -13 0 -13.0 -0.0 13.0 -1034.1889999999999 0.0 13.0 0.0 -10.175 -13.0 2.0215384615384617 -13.0 0.0 -13 -NULL 23 1969-12-31 16:00:06.391 23.0 NULL -23 23 0 23.0 0.0 -23.0 1829.719 0.0 -23.0 0.0 -10.175 23.0 -1.142608695652174 23.0 0.0 23 -true 42 1969-12-31 16:00:00.007 42.0 821UdmGbkEf4j -42 42 0 42.0 0.0 -42.0 3341.2259999999997 0.0 -42.0 0.0 -10.175 42.0 -0.6257142857142858 42.0 0.0 42 -NULL 43 1969-12-31 16:00:14.685 43.0 NULL -43 43 0 43.0 0.0 -43.0 3420.779 0.0 -43.0 0.0 -10.175 43.0 -0.6111627906976744 43.0 0.0 43 -true -27 1969-12-31 16:00:07.616 -27.0 cvLH6Eat2yFsyy7p 27 -27 0 -27.0 -0.0 27.0 -2147.931 0.0 27.0 0.0 -10.175 -27.0 0.9733333333333334 -27.0 0.0 -27 -true -54 1969-12-31 16:00:00.043 -54.0 821UdmGbkEf4j 54 -54 0 -54.0 -0.0 54.0 -4295.862 0.0 54.0 0.0 -10.175 -54.0 0.4866666666666667 -54.0 0.0 -54 -true -10 1969-12-31 16:00:01.784 -10.0 1cGVWH7n1QU 10 -10 0 -10.0 -0.0 10.0 -795.53 0.0 10.0 0.0 -10.175 -10.0 2.628 -10.0 0.0 -10 -NULL 3 1969-12-31 16:00:13.064 3.0 NULL -3 3 0 3.0 0.0 -3.0 238.659 0.0 -3.0 0.0 -10.175 3.0 -8.76 3.0 0.0 3 -NULL 49 1969-12-31 16:00:11.333 49.0 NULL -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 -NULL 54 1969-12-31 16:00:05.43 54.0 NULL -54 54 0 54.0 0.0 -54.0 4295.862 0.0 -54.0 0.0 -10.175 54.0 -0.4866666666666667 54.0 0.0 54 -NULL 1 1969-12-31 16:00:13.509 1.0 NULL -1 1 0 1.0 0.0 -1.0 79.553 0.0 -1.0 0.0 -10.175 1.0 -26.28 1.0 0.0 1 -true -61 1969-12-31 16:00:10.713 -61.0 cvLH6Eat2yFsyy7p 61 -61 0 -61.0 -0.0 61.0 -4852.733 0.0 61.0 0.0 -10.175 -61.0 0.4308196721311476 -61.0 0.0 -61 -NULL -23 1969-12-31 16:00:12.663 -23.0 NULL 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -true 18 1969-12-31 16:00:09.509 18.0 1cGVWH7n1QU -18 18 0 18.0 0.0 -18.0 1431.954 0.0 -18.0 0.0 -10.175 18.0 -1.46 18.0 0.0 18 -true -26 1969-12-31 16:00:13.49 -26.0 1cGVWH7n1QU 26 -26 0 -26.0 -0.0 26.0 -2068.3779999999997 0.0 26.0 0.0 -10.175 -26.0 1.0107692307692309 -26.0 0.0 -26 -NULL 57 1969-12-31 16:00:12.523 57.0 NULL -57 57 0 57.0 0.0 -57.0 4534.521 0.0 -57.0 0.0 -10.175 57.0 -0.4610526315789474 57.0 0.0 57 -NULL 58 1969-12-31 16:00:04.031 58.0 NULL -58 58 0 58.0 0.0 -58.0 4614.074 0.0 -58.0 0.0 -10.175 58.0 -0.4531034482758621 58.0 0.0 58 -true 41 1969-12-31 16:00:04.128 41.0 cvLH6Eat2yFsyy7p -41 41 0 41.0 0.0 -41.0 3261.673 0.0 -41.0 0.0 -10.175 41.0 -0.6409756097560976 41.0 0.0 41 -NULL 49 1969-12-31 16:00:00.754 49.0 NULL -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 -NULL 36 1969-12-31 16:00:10.467 36.0 NULL -36 36 0 36.0 0.0 -36.0 2863.908 0.0 -36.0 0.0 -10.175 36.0 -0.73 36.0 0.0 36 -NULL -46 1969-12-31 16:00:05.229 -46.0 NULL 46 -46 0 -46.0 -0.0 46.0 -3659.438 0.0 46.0 0.0 -10.175 -46.0 0.571304347826087 -46.0 0.0 -46 -NULL -19 1969-12-31 16:00:09.061 -19.0 NULL 19 -19 0 -19.0 -0.0 19.0 -1511.507 0.0 19.0 0.0 -10.175 -19.0 1.3831578947368421 -19.0 0.0 -19 -NULL 11 1969-12-31 16:00:08.27 11.0 NULL -11 11 0 11.0 0.0 -11.0 875.083 0.0 -11.0 0.0 -10.175 11.0 -2.389090909090909 11.0 0.0 11 -true 50 1969-12-31 16:00:16.338 50.0 821UdmGbkEf4j -50 50 0 50.0 0.0 -50.0 3977.6499999999996 0.0 -50.0 0.0 -10.175 50.0 -0.5256000000000001 50.0 0.0 50 -NULL -18 1969-12-31 16:00:06.848 -18.0 NULL 18 -18 0 -18.0 -0.0 18.0 -1431.954 0.0 18.0 0.0 -10.175 -18.0 1.46 -18.0 0.0 -18 -NULL -34 1969-12-31 16:00:00.357 -34.0 NULL 34 -34 0 -34.0 -0.0 34.0 -2704.8019999999997 0.0 34.0 0.0 -10.175 -34.0 0.7729411764705882 -34.0 0.0 -34 -NULL -10 1969-12-31 16:00:11.061 -10.0 NULL 10 -10 0 -10.0 -0.0 10.0 -795.53 0.0 10.0 0.0 -10.175 -10.0 2.628 -10.0 0.0 -10 -true -41 1969-12-31 16:00:04.781 -41.0 1cGVWH7n1QU 41 -41 0 -41.0 -0.0 41.0 -3261.673 0.0 41.0 0.0 -10.175 -41.0 0.6409756097560976 -41.0 0.0 -41 -NULL -18 1969-12-31 16:00:05.241 -18.0 NULL 18 -18 0 -18.0 -0.0 18.0 -1431.954 0.0 18.0 0.0 -10.175 -18.0 1.46 -18.0 0.0 -18 -NULL 48 1969-12-31 16:00:10.548 48.0 NULL -48 48 0 48.0 0.0 -48.0 3818.544 0.0 -48.0 0.0 -10.175 48.0 -0.5475 48.0 0.0 48 -true 26 1969-12-31 16:00:00.479 26.0 cvLH6Eat2yFsyy7p -26 26 0 26.0 0.0 -26.0 2068.3779999999997 0.0 -26.0 0.0 -10.175 26.0 -1.0107692307692309 26.0 0.0 26 -true -24 1969-12-31 16:00:14.987 -24.0 1cGVWH7n1QU 24 -24 0 -24.0 -0.0 24.0 -1909.272 0.0 24.0 0.0 -10.175 -24.0 1.095 -24.0 0.0 -24 -NULL -22 1969-12-31 16:00:05.218 -22.0 NULL 22 -22 0 -22.0 -0.0 22.0 -1750.166 0.0 22.0 0.0 -10.175 -22.0 1.1945454545454546 -22.0 0.0 -22 -NULL 25 1969-12-31 16:00:03.855 25.0 NULL -25 25 0 25.0 0.0 -25.0 1988.8249999999998 0.0 -25.0 0.0 -10.175 25.0 -1.0512000000000001 25.0 0.0 25 -true 2 1969-12-31 16:00:15.103 2.0 cvLH6Eat2yFsyy7p -2 2 0 2.0 0.0 -2.0 159.106 0.0 -2.0 0.0 -10.175 2.0 -13.14 2.0 0.0 2 -true -1 1969-12-31 16:00:04.956 -1.0 821UdmGbkEf4j 1 -1 0 -1.0 -0.0 1.0 -79.553 0.0 1.0 0.0 -10.175 -1.0 26.28 -1.0 0.0 -1 -true -47 1969-12-31 16:00:09.447 -47.0 cvLH6Eat2yFsyy7p 47 -47 0 -47.0 -0.0 47.0 -3738.991 0.0 47.0 0.0 -10.175 -47.0 0.5591489361702128 -47.0 0.0 -47 -NULL 48 1969-12-31 16:00:06.529 48.0 NULL -48 48 0 48.0 0.0 -48.0 3818.544 0.0 -48.0 0.0 -10.175 48.0 -0.5475 48.0 0.0 48 -true 43 1969-12-31 16:00:00.74 43.0 1cGVWH7n1QU -43 43 0 43.0 0.0 -43.0 3420.779 0.0 -43.0 0.0 -10.175 43.0 -0.6111627906976744 43.0 0.0 43 -NULL 17 1969-12-31 16:00:07.296 17.0 NULL -17 17 0 17.0 0.0 -17.0 1352.4009999999998 0.0 -17.0 0.0 -10.175 17.0 -1.5458823529411765 17.0 0.0 17 -NULL 15 1969-12-31 16:00:13.404 15.0 NULL -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -true 44 1969-12-31 16:00:11.625 44.0 cvLH6Eat2yFsyy7p -44 44 0 44.0 0.0 -44.0 3500.332 0.0 -44.0 0.0 -10.175 44.0 -0.5972727272727273 44.0 0.0 44 -NULL -14 1969-12-31 16:00:07.6 -14.0 NULL 14 -14 0 -14.0 -0.0 14.0 -1113.742 0.0 14.0 0.0 -10.175 -14.0 1.8771428571428572 -14.0 0.0 -14 -NULL 57 1969-12-31 16:00:04.207 57.0 NULL -57 57 0 57.0 0.0 -57.0 4534.521 0.0 -57.0 0.0 -10.175 57.0 -0.4610526315789474 57.0 0.0 57 -true 62 1969-12-31 16:00:11.073 62.0 1cGVWH7n1QU -62 62 0 62.0 0.0 -62.0 4932.286 0.0 -62.0 0.0 -10.175 62.0 -0.4238709677419355 62.0 0.0 62 -true 55 1969-12-31 16:00:12.871 55.0 1cGVWH7n1QU -55 55 0 55.0 0.0 -55.0 4375.415 0.0 -55.0 0.0 -10.175 55.0 -0.47781818181818186 55.0 0.0 55 -NULL 31 1969-12-31 16:00:13.816 31.0 NULL -31 31 0 31.0 0.0 -31.0 2466.143 0.0 -31.0 0.0 -10.175 31.0 -0.847741935483871 31.0 0.0 31 -NULL -61 1969-12-31 16:00:04.977 -61.0 NULL 61 -61 0 -61.0 -0.0 61.0 -4852.733 0.0 61.0 0.0 -10.175 -61.0 0.4308196721311476 -61.0 0.0 -61 -true -64 1969-12-31 16:00:10.273 -64.0 821UdmGbkEf4j 64 -64 0 -64.0 -0.0 64.0 -5091.392 0.0 64.0 0.0 -10.175 -64.0 0.410625 -64.0 0.0 -64 -NULL -28 1969-12-31 16:00:12.796 -28.0 NULL 28 -28 0 -28.0 -0.0 28.0 -2227.484 0.0 28.0 0.0 -10.175 -28.0 0.9385714285714286 -28.0 0.0 -28 -true -50 1969-12-31 16:00:06.167 -50.0 1cGVWH7n1QU 50 -50 0 -50.0 -0.0 50.0 -3977.6499999999996 0.0 50.0 0.0 -10.175 -50.0 0.5256000000000001 -50.0 0.0 -50 -true 5 1969-12-31 16:00:10.233 5.0 1cGVWH7n1QU -5 5 0 5.0 0.0 -5.0 397.765 0.0 -5.0 0.0 -10.175 5.0 -5.256 5.0 0.0 5 -true 62 1969-12-31 16:00:02.922 62.0 821UdmGbkEf4j -62 62 0 62.0 0.0 -62.0 4932.286 0.0 -62.0 0.0 -10.175 62.0 -0.4238709677419355 62.0 0.0 62 -true 11 1969-12-31 16:00:02.192 11.0 cvLH6Eat2yFsyy7p -11 11 0 11.0 0.0 -11.0 875.083 0.0 -11.0 0.0 -10.175 11.0 -2.389090909090909 11.0 0.0 11 -true 62 1969-12-31 16:00:08.785 62.0 cvLH6Eat2yFsyy7p -62 62 0 62.0 0.0 -62.0 4932.286 0.0 -62.0 0.0 -10.175 62.0 -0.4238709677419355 62.0 0.0 62 -NULL 61 1969-12-31 16:00:02.361 61.0 NULL -61 61 0 61.0 0.0 -61.0 4852.733 0.0 -61.0 0.0 -10.175 61.0 -0.4308196721311476 61.0 0.0 61 -true 41 1969-12-31 16:00:05.897 41.0 1cGVWH7n1QU -41 41 0 41.0 0.0 -41.0 3261.673 0.0 -41.0 0.0 -10.175 41.0 -0.6409756097560976 41.0 0.0 41 -NULL 38 1969-12-31 16:00:16.165 38.0 NULL -38 38 0 38.0 0.0 -38.0 3023.014 0.0 -38.0 0.0 -10.175 38.0 -0.6915789473684211 38.0 0.0 38 -true 47 1969-12-31 16:00:13.684 47.0 821UdmGbkEf4j -47 47 0 47.0 0.0 -47.0 3738.991 0.0 -47.0 0.0 -10.175 47.0 -0.5591489361702128 47.0 0.0 47 -true -29 1969-12-31 16:00:04.544 -29.0 cvLH6Eat2yFsyy7p 29 -29 0 -29.0 -0.0 29.0 -2307.037 0.0 29.0 0.0 -10.175 -29.0 0.9062068965517242 -29.0 0.0 -29 -true 19 1969-12-31 16:00:11.333 19.0 1cGVWH7n1QU -19 19 0 19.0 0.0 -19.0 1511.507 0.0 -19.0 0.0 -10.175 19.0 -1.3831578947368421 19.0 0.0 19 -NULL 30 1969-12-31 16:00:02.401 30.0 NULL -30 30 0 30.0 0.0 -30.0 2386.59 0.0 -30.0 0.0 -10.175 30.0 -0.876 30.0 0.0 30 -NULL -29 1969-12-31 16:00:09.485 -29.0 NULL 29 -29 0 -29.0 -0.0 29.0 -2307.037 0.0 29.0 0.0 -10.175 -29.0 0.9062068965517242 -29.0 0.0 -29 -true 41 1969-12-31 16:00:12.235 41.0 cvLH6Eat2yFsyy7p -41 41 0 41.0 0.0 -41.0 3261.673 0.0 -41.0 0.0 -10.175 41.0 -0.6409756097560976 41.0 0.0 41 -NULL 36 1969-12-31 16:00:12.979 36.0 NULL -36 36 0 36.0 0.0 -36.0 2863.908 0.0 -36.0 0.0 -10.175 36.0 -0.73 36.0 0.0 36 -true 38 1969-12-31 16:00:31.808 38.0 1cGVWH7n1QU -38 38 0 38.0 0.0 -38.0 3023.014 0.0 -38.0 0.0 -10.175 38.0 -0.6915789473684211 38.0 0.0 38 -NULL -3 1969-12-31 16:00:09.434 -3.0 NULL 3 -3 0 -3.0 -0.0 3.0 -238.659 0.0 3.0 0.0 -10.175 -3.0 8.76 -3.0 0.0 -3 -true 30 1969-12-31 16:00:10.538 30.0 cvLH6Eat2yFsyy7p -30 30 0 30.0 0.0 -30.0 2386.59 0.0 -30.0 0.0 -10.175 30.0 -0.876 30.0 0.0 30 -NULL 15 1969-12-31 16:00:14.98 15.0 NULL -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -NULL 12 1969-12-31 16:00:00.911 12.0 NULL -12 12 0 12.0 0.0 -12.0 954.636 0.0 -12.0 0.0 -10.175 12.0 -2.19 12.0 0.0 12 -NULL 2 1969-12-31 16:00:15.965 2.0 NULL -2 2 0 2.0 0.0 -2.0 159.106 0.0 -2.0 0.0 -10.175 2.0 -13.14 2.0 0.0 2 -true 60 1969-12-31 16:00:00.865 60.0 821UdmGbkEf4j -60 60 0 60.0 0.0 -60.0 4773.18 0.0 -60.0 0.0 -10.175 60.0 -0.438 60.0 0.0 60 -NULL -45 1969-12-31 16:00:12.685 -45.0 NULL 45 -45 0 -45.0 -0.0 45.0 -3579.8849999999998 0.0 45.0 0.0 -10.175 -45.0 0.5840000000000001 -45.0 0.0 -45 -NULL -48 1969-12-31 16:00:06.198 -48.0 NULL 48 -48 0 -48.0 -0.0 48.0 -3818.544 0.0 48.0 0.0 -10.175 -48.0 0.5475 -48.0 0.0 -48 -true -15 1969-12-31 16:00:07.482 -15.0 cvLH6Eat2yFsyy7p 15 -15 0 -15.0 -0.0 15.0 -1193.295 0.0 15.0 0.0 -10.175 -15.0 1.752 -15.0 0.0 -15 -NULL 47 1969-12-31 16:00:02.32 47.0 NULL -47 47 0 47.0 0.0 -47.0 3738.991 0.0 -47.0 0.0 -10.175 47.0 -0.5591489361702128 47.0 0.0 47 -NULL -48 1969-12-31 16:00:04.552 -48.0 NULL 48 -48 0 -48.0 -0.0 48.0 -3818.544 0.0 48.0 0.0 -10.175 -48.0 0.5475 -48.0 0.0 -48 -true 29 1969-12-31 16:00:01.202 29.0 cvLH6Eat2yFsyy7p -29 29 0 29.0 0.0 -29.0 2307.037 0.0 -29.0 0.0 -10.175 29.0 -0.9062068965517242 29.0 0.0 29 -true 4 1969-12-31 16:00:06.666 4.0 821UdmGbkEf4j -4 4 0 4.0 0.0 -4.0 318.212 0.0 -4.0 0.0 -10.175 4.0 -6.57 4.0 0.0 4 -true -43 1969-12-31 16:00:02.601 -43.0 821UdmGbkEf4j 43 -43 0 -43.0 -0.0 43.0 -3420.779 0.0 43.0 0.0 -10.175 -43.0 0.6111627906976744 -43.0 0.0 -43 -NULL 44 1969-12-31 16:00:16.279 44.0 NULL -44 44 0 44.0 0.0 -44.0 3500.332 0.0 -44.0 0.0 -10.175 44.0 -0.5972727272727273 44.0 0.0 44 -true -18 1969-12-31 16:00:07.777 -18.0 821UdmGbkEf4j 18 -18 0 -18.0 -0.0 18.0 -1431.954 0.0 18.0 0.0 -10.175 -18.0 1.46 -18.0 0.0 -18 -true 33 1969-12-31 16:00:00.323 79.553 1cGVWH7n1QU -33 33 0 79.5530014038086 0.0 -79.5530014038086 6328.679920677185 0.0 -79.5530014038086 0.0 -10.175 79.5530014038086 -0.33034580136836733 79.553 0.0 33 -true -29 1969-12-31 16:00:01.752 -29.0 821UdmGbkEf4j 29 -29 0 -29.0 -0.0 29.0 -2307.037 0.0 29.0 0.0 -10.175 -29.0 0.9062068965517242 -29.0 0.0 -29 -NULL 26 1969-12-31 16:00:04.233 26.0 NULL -26 26 0 26.0 0.0 -26.0 2068.3779999999997 0.0 -26.0 0.0 -10.175 26.0 -1.0107692307692309 26.0 0.0 26 -true 27 1969-12-31 16:00:00.547 27.0 821UdmGbkEf4j -27 27 0 27.0 0.0 -27.0 2147.931 0.0 -27.0 0.0 -10.175 27.0 -0.9733333333333334 27.0 0.0 27 -NULL -39 1969-12-31 16:00:13.046 -39.0 NULL 39 -39 0 -39.0 -0.0 39.0 -3102.567 0.0 39.0 0.0 -10.175 -39.0 0.6738461538461539 -39.0 0.0 -39 -NULL 29 1969-12-31 16:00:16.19 29.0 NULL -29 29 0 29.0 0.0 -29.0 2307.037 0.0 -29.0 0.0 -10.175 29.0 -0.9062068965517242 29.0 0.0 29 -NULL -40 1969-12-31 16:00:13.221 -40.0 NULL 40 -40 0 -40.0 -0.0 40.0 -3182.12 0.0 40.0 0.0 -10.175 -40.0 0.657 -40.0 0.0 -40 -NULL -44 1969-12-31 16:00:08.12 -44.0 NULL 44 -44 0 -44.0 -0.0 44.0 -3500.332 0.0 44.0 0.0 -10.175 -44.0 0.5972727272727273 -44.0 0.0 -44 -true 49 1969-12-31 16:00:04.907 49.0 1cGVWH7n1QU -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 -NULL -62 1969-12-31 16:00:10.225 -62.0 NULL 62 -62 0 -62.0 -0.0 62.0 -4932.286 0.0 62.0 0.0 -10.175 -62.0 0.4238709677419355 -62.0 0.0 -62 -true 7 1969-12-31 16:00:09.828 7.0 821UdmGbkEf4j -7 7 0 7.0 0.0 -7.0 556.871 0.0 -7.0 0.0 -10.175 7.0 -3.7542857142857144 7.0 0.0 7 -NULL 25 1969-12-31 16:00:12.163 25.0 NULL -25 25 0 25.0 0.0 -25.0 1988.8249999999998 0.0 -25.0 0.0 -10.175 25.0 -1.0512000000000001 25.0 0.0 25 -NULL 22 1969-12-31 16:00:00.621 22.0 NULL -22 22 0 22.0 0.0 -22.0 1750.166 0.0 -22.0 0.0 -10.175 22.0 -1.1945454545454546 22.0 0.0 22 -true 9 1969-12-31 16:00:09.444 9.0 821UdmGbkEf4j -9 9 0 9.0 0.0 -9.0 715.977 0.0 -9.0 0.0 -10.175 9.0 -2.92 9.0 0.0 9 -true 16 1969-12-31 16:00:16.227 16.0 821UdmGbkEf4j -16 16 0 16.0 0.0 -16.0 1272.848 0.0 -16.0 0.0 -10.175 16.0 -1.6425 16.0 0.0 16 -NULL -57 1969-12-31 16:00:06.664 -57.0 NULL 57 -57 0 -57.0 -0.0 57.0 -4534.521 0.0 57.0 0.0 -10.175 -57.0 0.4610526315789474 -57.0 0.0 -57 -NULL -10 1969-12-31 16:00:03.248 -10.0 NULL 10 -10 0 -10.0 -0.0 10.0 -795.53 0.0 10.0 0.0 -10.175 -10.0 2.628 -10.0 0.0 -10 -NULL -31 1969-12-31 16:00:05.617 -31.0 NULL 31 -31 0 -31.0 -0.0 31.0 -2466.143 0.0 31.0 0.0 -10.175 -31.0 0.847741935483871 -31.0 0.0 -31 -NULL -64 1969-12-31 16:00:00.199 -64.0 NULL 64 -64 0 -64.0 -0.0 64.0 -5091.392 0.0 64.0 0.0 -10.175 -64.0 0.410625 -64.0 0.0 -64 -true 5 1969-12-31 16:00:05.78 5.0 cvLH6Eat2yFsyy7p -5 5 0 5.0 0.0 -5.0 397.765 0.0 -5.0 0.0 -10.175 5.0 -5.256 5.0 0.0 5 -NULL 3 1969-12-31 16:00:05.162 3.0 NULL -3 3 0 3.0 0.0 -3.0 238.659 0.0 -3.0 0.0 -10.175 3.0 -8.76 3.0 0.0 3 -NULL -17 1969-12-31 16:00:10.939 -17.0 NULL 17 -17 0 -17.0 -0.0 17.0 -1352.4009999999998 0.0 17.0 0.0 -10.175 -17.0 1.5458823529411765 -17.0 0.0 -17 -NULL -9 1969-12-31 16:00:07.651 -9.0 NULL 9 -9 0 -9.0 -0.0 9.0 -715.977 0.0 9.0 0.0 -10.175 -9.0 2.92 -9.0 0.0 -9 -true 23 1969-12-31 16:00:05.929 23.0 821UdmGbkEf4j -23 23 0 23.0 0.0 -23.0 1829.719 0.0 -23.0 0.0 -10.175 23.0 -1.142608695652174 23.0 0.0 23 -true 34 1969-12-31 16:00:01.931 34.0 821UdmGbkEf4j -34 34 0 34.0 0.0 -34.0 2704.8019999999997 0.0 -34.0 0.0 -10.175 34.0 -0.7729411764705882 34.0 0.0 34 -NULL 2 1969-12-31 16:00:03.369 2.0 NULL -2 2 0 2.0 0.0 -2.0 159.106 0.0 -2.0 0.0 -10.175 2.0 -13.14 2.0 0.0 2 -true -1 1969-12-31 16:00:15.242 -1.0 cvLH6Eat2yFsyy7p 1 -1 0 -1.0 -0.0 1.0 -79.553 0.0 1.0 0.0 -10.175 -1.0 26.28 -1.0 0.0 -1 -true 52 1969-12-31 16:00:12.652 52.0 821UdmGbkEf4j -52 52 0 52.0 0.0 -52.0 4136.755999999999 0.0 -52.0 0.0 -10.175 52.0 -0.5053846153846154 52.0 0.0 52 -true 59 1969-12-31 16:00:06.7 59.0 821UdmGbkEf4j -59 59 0 59.0 0.0 -59.0 4693.6269999999995 0.0 -59.0 0.0 -10.175 59.0 -0.44542372881355935 59.0 0.0 59 -NULL -24 1969-12-31 16:00:08.839 -24.0 NULL 24 -24 0 -24.0 -0.0 24.0 -1909.272 0.0 24.0 0.0 -10.175 -24.0 1.095 -24.0 0.0 -24 -true -13 1969-12-31 16:00:01.828 -13.0 1cGVWH7n1QU 13 -13 0 -13.0 -0.0 13.0 -1034.1889999999999 0.0 13.0 0.0 -10.175 -13.0 2.0215384615384617 -13.0 0.0 -13 -NULL -55 1969-12-31 16:00:03.737 -55.0 NULL 55 -55 0 -55.0 -0.0 55.0 -4375.415 0.0 55.0 0.0 -10.175 -55.0 0.47781818181818186 -55.0 0.0 -55 -true -29 1969-12-31 16:00:05.887 -29.0 1cGVWH7n1QU 29 -29 0 -29.0 -0.0 29.0 -2307.037 0.0 29.0 0.0 -10.175 -29.0 0.9062068965517242 -29.0 0.0 -29 -NULL 29 1969-12-31 16:00:15.582 29.0 NULL -29 29 0 29.0 0.0 -29.0 2307.037 0.0 -29.0 0.0 -10.175 29.0 -0.9062068965517242 29.0 0.0 29 -NULL 15 1969-12-31 16:00:00.951 15.0 NULL -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -NULL -20 1969-12-31 16:00:10.551 -20.0 NULL 20 -20 0 -20.0 -0.0 20.0 -1591.06 0.0 20.0 0.0 -10.175 -20.0 1.314 -20.0 0.0 -20 -true -38 1969-12-31 16:00:14.195 -38.0 cvLH6Eat2yFsyy7p 38 -38 0 -38.0 -0.0 38.0 -3023.014 0.0 38.0 0.0 -10.175 -38.0 0.6915789473684211 -38.0 0.0 -38 -true -53 1969-12-31 16:00:01.437 -53.0 821UdmGbkEf4j 53 -53 0 -53.0 -0.0 53.0 -4216.309 0.0 53.0 0.0 -10.175 -53.0 0.4958490566037736 -53.0 0.0 -53 -true -26 1969-12-31 16:00:03.086 -26.0 1cGVWH7n1QU 26 -26 0 -26.0 -0.0 26.0 -2068.3779999999997 0.0 26.0 0.0 -10.175 -26.0 1.0107692307692309 -26.0 0.0 -26 -NULL 54 1969-12-31 16:00:08.793 54.0 NULL -54 54 0 54.0 0.0 -54.0 4295.862 0.0 -54.0 0.0 -10.175 54.0 -0.4866666666666667 54.0 0.0 54 -NULL -43 1969-12-31 16:00:14.048 -43.0 NULL 43 -43 0 -43.0 -0.0 43.0 -3420.779 0.0 43.0 0.0 -10.175 -43.0 0.6111627906976744 -43.0 0.0 -43 -true 45 1969-12-31 16:00:11.522 45.0 1cGVWH7n1QU -45 45 0 45.0 0.0 -45.0 3579.8849999999998 0.0 -45.0 0.0 -10.175 45.0 -0.5840000000000001 45.0 0.0 45 -NULL 47 1969-12-31 16:00:02 47.0 NULL -47 47 0 47.0 0.0 -47.0 3738.991 0.0 -47.0 0.0 -10.175 47.0 -0.5591489361702128 47.0 0.0 47 -true -11 1969-12-31 16:00:01.785 -11.0 cvLH6Eat2yFsyy7p 11 -11 0 -11.0 -0.0 11.0 -875.083 0.0 11.0 0.0 -10.175 -11.0 2.389090909090909 -11.0 0.0 -11 -true -59 1969-12-31 16:00:16.125 -59.0 cvLH6Eat2yFsyy7p 59 -59 0 -59.0 -0.0 59.0 -4693.6269999999995 0.0 59.0 0.0 -10.175 -59.0 0.44542372881355935 -59.0 0.0 -59 -true -43 1969-12-31 16:00:12.637 -43.0 1cGVWH7n1QU 43 -43 0 -43.0 -0.0 43.0 -3420.779 0.0 43.0 0.0 -10.175 -43.0 0.6111627906976744 -43.0 0.0 -43 -NULL -49 1969-12-31 16:00:01.458 -49.0 NULL 49 -49 0 -49.0 -0.0 49.0 -3898.0969999999998 0.0 49.0 0.0 -10.175 -49.0 0.5363265306122449 -49.0 0.0 -49 -true -35 1969-12-31 16:00:02.424 -35.0 1cGVWH7n1QU 35 -35 0 -35.0 -0.0 35.0 -2784.355 0.0 35.0 0.0 -10.175 -35.0 0.7508571428571429 -35.0 0.0 -35 -true 40 1969-12-31 16:00:08.242 40.0 1cGVWH7n1QU -40 40 0 40.0 0.0 -40.0 3182.12 0.0 -40.0 0.0 -10.175 40.0 -0.657 40.0 0.0 40 -NULL 2 1969-12-31 16:00:03.672 2.0 NULL -2 2 0 2.0 0.0 -2.0 159.106 0.0 -2.0 0.0 -10.175 2.0 -13.14 2.0 0.0 2 -true 53 1969-12-31 16:00:05.897 53.0 1cGVWH7n1QU -53 53 0 53.0 0.0 -53.0 4216.309 0.0 -53.0 0.0 -10.175 53.0 -0.4958490566037736 53.0 0.0 53 -NULL 18 1969-12-31 16:00:00.459 18.0 NULL -18 18 0 18.0 0.0 -18.0 1431.954 0.0 -18.0 0.0 -10.175 18.0 -1.46 18.0 0.0 18 -true 46 1969-12-31 16:00:05.211 46.0 821UdmGbkEf4j -46 46 0 46.0 0.0 -46.0 3659.438 0.0 -46.0 0.0 -10.175 46.0 -0.571304347826087 46.0 0.0 46 -NULL -22 1969-12-31 16:00:02.227 -22.0 NULL 22 -22 0 -22.0 -0.0 22.0 -1750.166 0.0 22.0 0.0 -10.175 -22.0 1.1945454545454546 -22.0 0.0 -22 -NULL 7 1969-12-31 16:00:14.872 7.0 NULL -7 7 0 7.0 0.0 -7.0 556.871 0.0 -7.0 0.0 -10.175 7.0 -3.7542857142857144 7.0 0.0 7 -NULL 53 1969-12-31 16:00:15.401 53.0 NULL -53 53 0 53.0 0.0 -53.0 4216.309 0.0 -53.0 0.0 -10.175 53.0 -0.4958490566037736 53.0 0.0 53 -true 14 1969-12-31 16:00:00.938 14.0 cvLH6Eat2yFsyy7p -14 14 0 14.0 0.0 -14.0 1113.742 0.0 -14.0 0.0 -10.175 14.0 -1.8771428571428572 14.0 0.0 14 -true 59 1969-12-31 16:00:16.24 59.0 821UdmGbkEf4j -59 59 0 59.0 0.0 -59.0 4693.6269999999995 0.0 -59.0 0.0 -10.175 59.0 -0.44542372881355935 59.0 0.0 59 -true 3 1969-12-31 16:00:03.119 3.0 cvLH6Eat2yFsyy7p -3 3 0 3.0 0.0 -3.0 238.659 0.0 -3.0 0.0 -10.175 3.0 -8.76 3.0 0.0 3 -NULL 18 1969-12-31 16:00:11.197 18.0 NULL -18 18 0 18.0 0.0 -18.0 1431.954 0.0 -18.0 0.0 -10.175 18.0 -1.46 18.0 0.0 18 -true 6 1969-12-31 16:00:04.901 6.0 1cGVWH7n1QU -6 6 0 6.0 0.0 -6.0 477.318 0.0 -6.0 0.0 -10.175 6.0 -4.38 6.0 0.0 6 -true 14 1969-12-31 16:00:03.666 14.0 821UdmGbkEf4j -14 14 0 14.0 0.0 -14.0 1113.742 0.0 -14.0 0.0 -10.175 14.0 -1.8771428571428572 14.0 0.0 14 -NULL -49 1969-12-31 16:00:01.336 -49.0 NULL 49 -49 0 -49.0 -0.0 49.0 -3898.0969999999998 0.0 49.0 0.0 -10.175 -49.0 0.5363265306122449 -49.0 0.0 -49 -true -62 1969-12-31 16:00:01.419 -62.0 821UdmGbkEf4j 62 -62 0 -62.0 -0.0 62.0 -4932.286 0.0 62.0 0.0 -10.175 -62.0 0.4238709677419355 -62.0 0.0 -62 -NULL 47 1969-12-31 16:00:06.256 47.0 NULL -47 47 0 47.0 0.0 -47.0 3738.991 0.0 -47.0 0.0 -10.175 47.0 -0.5591489361702128 47.0 0.0 47 -NULL -61 1969-12-31 16:00:06.848 -61.0 NULL 61 -61 0 -61.0 -0.0 61.0 -4852.733 0.0 61.0 0.0 -10.175 -61.0 0.4308196721311476 -61.0 0.0 -61 -NULL 62 1969-12-31 16:00:02.38 62.0 NULL -62 62 0 62.0 0.0 -62.0 4932.286 0.0 -62.0 0.0 -10.175 62.0 -0.4238709677419355 62.0 0.0 62 -true -48 1969-12-31 16:00:00.309 -48.0 1cGVWH7n1QU 48 -48 0 -48.0 -0.0 48.0 -3818.544 0.0 48.0 0.0 -10.175 -48.0 0.5475 -48.0 0.0 -48 -true -5 1969-12-31 16:00:16.337 -5.0 821UdmGbkEf4j 5 -5 0 -5.0 -0.0 5.0 -397.765 0.0 5.0 0.0 -10.175 -5.0 5.256 -5.0 0.0 -5 -true -46 1969-12-31 16:00:01.562 -46.0 cvLH6Eat2yFsyy7p 46 -46 0 -46.0 -0.0 46.0 -3659.438 0.0 46.0 0.0 -10.175 -46.0 0.571304347826087 -46.0 0.0 -46 -NULL -15 1969-12-31 16:00:14.525 -15.0 NULL 15 -15 0 -15.0 -0.0 15.0 -1193.295 0.0 15.0 0.0 -10.175 -15.0 1.752 -15.0 0.0 -15 -NULL 13 1969-12-31 16:00:05.368 13.0 NULL -13 13 0 13.0 0.0 -13.0 1034.1889999999999 0.0 -13.0 0.0 -10.175 13.0 -2.0215384615384617 13.0 0.0 13 -NULL 50 1969-12-31 16:00:10.654 50.0 NULL -50 50 0 50.0 0.0 -50.0 3977.6499999999996 0.0 -50.0 0.0 -10.175 50.0 -0.5256000000000001 50.0 0.0 50 -true -27 1969-12-31 16:00:13.651 -27.0 1cGVWH7n1QU 27 -27 0 -27.0 -0.0 27.0 -2147.931 0.0 27.0 0.0 -10.175 -27.0 0.9733333333333334 -27.0 0.0 -27 -NULL 21 1969-12-31 16:00:03.4 21.0 NULL -21 21 0 21.0 0.0 -21.0 1670.6129999999998 0.0 -21.0 0.0 -10.175 21.0 -1.2514285714285716 21.0 0.0 21 -true -23 1969-12-31 16:00:00.707 -23.0 821UdmGbkEf4j 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -NULL -1 1969-12-31 16:00:12.403 -1.0 NULL 1 -1 0 -1.0 -0.0 1.0 -79.553 0.0 1.0 0.0 -10.175 -1.0 26.28 -1.0 0.0 -1 -true 56 1969-12-31 16:00:00.192 56.0 821UdmGbkEf4j -56 56 0 56.0 0.0 -56.0 4454.968 0.0 -56.0 0.0 -10.175 56.0 -0.4692857142857143 56.0 0.0 56 -NULL -51 1969-12-31 16:00:05.279 -51.0 NULL 51 -51 0 -51.0 -0.0 51.0 -4057.203 0.0 51.0 0.0 -10.175 -51.0 0.5152941176470588 -51.0 0.0 -51 -NULL -39 1969-12-31 16:00:00.553 -39.0 NULL 39 -39 0 -39.0 -0.0 39.0 -3102.567 0.0 39.0 0.0 -10.175 -39.0 0.6738461538461539 -39.0 0.0 -39 -NULL 25 1969-12-31 16:00:00.52 25.0 NULL -25 25 0 25.0 0.0 -25.0 1988.8249999999998 0.0 -25.0 0.0 -10.175 25.0 -1.0512000000000001 25.0 0.0 25 -true -12 1969-12-31 16:00:01.967 -12.0 1cGVWH7n1QU 12 -12 0 -12.0 -0.0 12.0 -954.636 0.0 12.0 0.0 -10.175 -12.0 2.19 -12.0 0.0 -12 -true 48 1969-12-31 16:00:13.232 48.0 cvLH6Eat2yFsyy7p -48 48 0 48.0 0.0 -48.0 3818.544 0.0 -48.0 0.0 -10.175 48.0 -0.5475 48.0 0.0 48 -true -59 1969-12-31 16:00:00.838 -59.0 cvLH6Eat2yFsyy7p 59 -59 0 -59.0 -0.0 59.0 -4693.6269999999995 0.0 59.0 0.0 -10.175 -59.0 0.44542372881355935 -59.0 0.0 -59 -NULL 29 1969-12-31 16:00:05.79 29.0 NULL -29 29 0 29.0 0.0 -29.0 2307.037 0.0 -29.0 0.0 -10.175 29.0 -0.9062068965517242 29.0 0.0 29 -NULL -19 1969-12-31 16:00:15.223 -19.0 NULL 19 -19 0 -19.0 -0.0 19.0 -1511.507 0.0 19.0 0.0 -10.175 -19.0 1.3831578947368421 -19.0 0.0 -19 -true -6 1969-12-31 16:00:07.442 -6.0 cvLH6Eat2yFsyy7p 6 -6 0 -6.0 -0.0 6.0 -477.318 0.0 6.0 0.0 -10.175 -6.0 4.38 -6.0 0.0 -6 -NULL 48 1969-12-31 16:00:08.445 48.0 NULL -48 48 0 48.0 0.0 -48.0 3818.544 0.0 -48.0 0.0 -10.175 48.0 -0.5475 48.0 0.0 48 -NULL 15 1969-12-31 16:00:04.585 15.0 NULL -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -NULL 44 1969-12-31 16:00:12.427 44.0 NULL -44 44 0 44.0 0.0 -44.0 3500.332 0.0 -44.0 0.0 -10.175 44.0 -0.5972727272727273 44.0 0.0 44 -true -45 1969-12-31 16:00:08.096 -45.0 821UdmGbkEf4j 45 -45 0 -45.0 -0.0 45.0 -3579.8849999999998 0.0 45.0 0.0 -10.175 -45.0 0.5840000000000001 -45.0 0.0 -45 -NULL 55 1969-12-31 16:00:14.709 55.0 NULL -55 55 0 55.0 0.0 -55.0 4375.415 0.0 -55.0 0.0 -10.175 55.0 -0.47781818181818186 55.0 0.0 55 -NULL -38 1969-12-31 16:00:01.282 -38.0 NULL 38 -38 0 -38.0 -0.0 38.0 -3023.014 0.0 38.0 0.0 -10.175 -38.0 0.6915789473684211 -38.0 0.0 -38 -true 48 1969-12-31 16:00:00.13 48.0 cvLH6Eat2yFsyy7p -48 48 0 48.0 0.0 -48.0 3818.544 0.0 -48.0 0.0 -10.175 48.0 -0.5475 48.0 0.0 48 -NULL 17 1969-12-31 16:00:10.701 17.0 NULL -17 17 0 17.0 0.0 -17.0 1352.4009999999998 0.0 -17.0 0.0 -10.175 17.0 -1.5458823529411765 17.0 0.0 17 -true 1 1969-12-31 16:00:11.658 1.0 cvLH6Eat2yFsyy7p -1 1 0 1.0 0.0 -1.0 79.553 0.0 -1.0 0.0 -10.175 1.0 -26.28 1.0 0.0 1 -NULL -51 1969-12-31 16:00:00.156 -51.0 NULL 51 -51 0 -51.0 -0.0 51.0 -4057.203 0.0 51.0 0.0 -10.175 -51.0 0.5152941176470588 -51.0 0.0 -51 -NULL -2 1969-12-31 16:00:03.198 -2.0 NULL 2 -2 0 -2.0 -0.0 2.0 -159.106 0.0 2.0 0.0 -10.175 -2.0 13.14 -2.0 0.0 -2 -true 31 1969-12-31 16:00:05.58 31.0 821UdmGbkEf4j -31 31 0 31.0 0.0 -31.0 2466.143 0.0 -31.0 0.0 -10.175 31.0 -0.847741935483871 31.0 0.0 31 -true 42 1969-12-31 16:00:08.737 42.0 1cGVWH7n1QU -42 42 0 42.0 0.0 -42.0 3341.2259999999997 0.0 -42.0 0.0 -10.175 42.0 -0.6257142857142858 42.0 0.0 42 -NULL 13 1969-12-31 16:00:10.573 13.0 NULL -13 13 0 13.0 0.0 -13.0 1034.1889999999999 0.0 -13.0 0.0 -10.175 13.0 -2.0215384615384617 13.0 0.0 13 -true -4 1969-12-31 16:00:11.875 -4.0 1cGVWH7n1QU 4 -4 0 -4.0 -0.0 4.0 -318.212 0.0 4.0 0.0 -10.175 -4.0 6.57 -4.0 0.0 -4 -true 35 1969-12-31 16:00:05.741 35.0 cvLH6Eat2yFsyy7p -35 35 0 35.0 0.0 -35.0 2784.355 0.0 -35.0 0.0 -10.175 35.0 -0.7508571428571429 35.0 0.0 35 -NULL -9 1969-12-31 16:00:13.651 -9.0 NULL 9 -9 0 -9.0 -0.0 9.0 -715.977 0.0 9.0 0.0 -10.175 -9.0 2.92 -9.0 0.0 -9 -true -10 1969-12-31 16:00:02.006 -10.0 821UdmGbkEf4j 10 -10 0 -10.0 -0.0 10.0 -795.53 0.0 10.0 0.0 -10.175 -10.0 2.628 -10.0 0.0 -10 -NULL -22 1969-12-31 16:00:07.379 -22.0 NULL 22 -22 0 -22.0 -0.0 22.0 -1750.166 0.0 22.0 0.0 -10.175 -22.0 1.1945454545454546 -22.0 0.0 -22 -NULL -33 1969-12-31 16:00:00.611 -33.0 NULL 33 -33 0 -33.0 -0.0 33.0 -2625.249 0.0 33.0 0.0 -10.175 -33.0 0.7963636363636364 -33.0 0.0 -33 -true -52 1969-12-31 16:00:15.752 -52.0 cvLH6Eat2yFsyy7p 52 -52 0 -52.0 -0.0 52.0 -4136.755999999999 0.0 52.0 0.0 -10.175 -52.0 0.5053846153846154 -52.0 0.0 -52 -true -40 1969-12-31 16:00:05.741 -40.0 1cGVWH7n1QU 40 -40 0 -40.0 -0.0 40.0 -3182.12 0.0 40.0 0.0 -10.175 -40.0 0.657 -40.0 0.0 -40 -NULL -59 1969-12-31 16:00:15.296 -59.0 NULL 59 -59 0 -59.0 -0.0 59.0 -4693.6269999999995 0.0 59.0 0.0 -10.175 -59.0 0.44542372881355935 -59.0 0.0 -59 -true -50 1969-12-31 16:00:08.354 -50.0 cvLH6Eat2yFsyy7p 50 -50 0 -50.0 -0.0 50.0 -3977.6499999999996 0.0 50.0 0.0 -10.175 -50.0 0.5256000000000001 -50.0 0.0 -50 -NULL 39 1969-12-31 16:00:06.149 39.0 NULL -39 39 0 39.0 0.0 -39.0 3102.567 0.0 -39.0 0.0 -10.175 39.0 -0.6738461538461539 39.0 0.0 39 -true -51 1969-12-31 16:00:10.827 -51.0 821UdmGbkEf4j 51 -51 0 -51.0 -0.0 51.0 -4057.203 0.0 51.0 0.0 -10.175 -51.0 0.5152941176470588 -51.0 0.0 -51 -true 20 1969-12-31 16:00:12.269 20.0 821UdmGbkEf4j -20 20 0 20.0 0.0 -20.0 1591.06 0.0 -20.0 0.0 -10.175 20.0 -1.314 20.0 0.0 20 -true -35 1969-12-31 16:00:07.511 -35.0 821UdmGbkEf4j 35 -35 0 -35.0 -0.0 35.0 -2784.355 0.0 35.0 0.0 -10.175 -35.0 0.7508571428571429 -35.0 0.0 -35 -NULL 18 1969-12-31 16:00:07.087 18.0 NULL -18 18 0 18.0 0.0 -18.0 1431.954 0.0 -18.0 0.0 -10.175 18.0 -1.46 18.0 0.0 18 -NULL 0 1969-12-31 16:00:04.055 0.0 NULL 0 0 0 0.0 0.0 -0.0 0.0 0.0 -0.0 0.0 -10.175 0.0 NULL 0.0 NULL 0 -NULL -23 1969-12-31 16:00:02.834 -23.0 NULL 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -NULL 7 1969-12-31 16:00:06.018 7.0 NULL -7 7 0 7.0 0.0 -7.0 556.871 0.0 -7.0 0.0 -10.175 7.0 -3.7542857142857144 7.0 0.0 7 -true 61 1969-12-31 16:00:11.737 61.0 cvLH6Eat2yFsyy7p -61 61 0 61.0 0.0 -61.0 4852.733 0.0 -61.0 0.0 -10.175 61.0 -0.4308196721311476 61.0 0.0 61 -NULL 29 1969-12-31 16:00:11.081 29.0 NULL -29 29 0 29.0 0.0 -29.0 2307.037 0.0 -29.0 0.0 -10.175 29.0 -0.9062068965517242 29.0 0.0 29 -true 27 1969-12-31 16:00:12.422 27.0 cvLH6Eat2yFsyy7p -27 27 0 27.0 0.0 -27.0 2147.931 0.0 -27.0 0.0 -10.175 27.0 -0.9733333333333334 27.0 0.0 27 -NULL 57 1969-12-31 16:00:14.076 57.0 NULL -57 57 0 57.0 0.0 -57.0 4534.521 0.0 -57.0 0.0 -10.175 57.0 -0.4610526315789474 57.0 0.0 57 -true -5 1969-12-31 16:00:05.4 -5.0 cvLH6Eat2yFsyy7p 5 -5 0 -5.0 -0.0 5.0 -397.765 0.0 5.0 0.0 -10.175 -5.0 5.256 -5.0 0.0 -5 -NULL -30 1969-12-31 16:00:01.165 -30.0 NULL 30 -30 0 -30.0 -0.0 30.0 -2386.59 0.0 30.0 0.0 -10.175 -30.0 0.876 -30.0 0.0 -30 -NULL -28 1969-12-31 16:00:11.236 -28.0 NULL 28 -28 0 -28.0 -0.0 28.0 -2227.484 0.0 28.0 0.0 -10.175 -28.0 0.9385714285714286 -28.0 0.0 -28 -NULL 8 1969-12-31 16:00:00.613 8.0 NULL -8 8 0 8.0 0.0 -8.0 636.424 0.0 -8.0 0.0 -10.175 8.0 -3.285 8.0 0.0 8 -NULL -48 1969-12-31 16:00:07.783 -48.0 NULL 48 -48 0 -48.0 -0.0 48.0 -3818.544 0.0 48.0 0.0 -10.175 -48.0 0.5475 -48.0 0.0 -48 -true -17 1969-12-31 16:00:15.987 -17.0 1cGVWH7n1QU 17 -17 0 -17.0 -0.0 17.0 -1352.4009999999998 0.0 17.0 0.0 -10.175 -17.0 1.5458823529411765 -17.0 0.0 -17 -NULL 11 1969-12-31 16:00:13.383 11.0 NULL -11 11 0 11.0 0.0 -11.0 875.083 0.0 -11.0 0.0 -10.175 11.0 -2.389090909090909 11.0 0.0 11 -NULL -46 1969-12-31 16:00:15.805 -46.0 NULL 46 -46 0 -46.0 -0.0 46.0 -3659.438 0.0 46.0 0.0 -10.175 -46.0 0.571304347826087 -46.0 0.0 -46 -NULL -44 1969-12-31 16:00:10.131 -44.0 NULL 44 -44 0 -44.0 -0.0 44.0 -3500.332 0.0 44.0 0.0 -10.175 -44.0 0.5972727272727273 -44.0 0.0 -44 -NULL 50 1969-12-31 16:00:08.38 50.0 NULL -50 50 0 50.0 0.0 -50.0 3977.6499999999996 0.0 -50.0 0.0 -10.175 50.0 -0.5256000000000001 50.0 0.0 50 -true 35 1969-12-31 16:00:03.768 35.0 cvLH6Eat2yFsyy7p -35 35 0 35.0 0.0 -35.0 2784.355 0.0 -35.0 0.0 -10.175 35.0 -0.7508571428571429 35.0 0.0 35 -true -21 1969-12-31 16:00:01.037 -21.0 1cGVWH7n1QU 21 -21 0 -21.0 -0.0 21.0 -1670.6129999999998 0.0 21.0 0.0 -10.175 -21.0 1.2514285714285716 -21.0 0.0 -21 -NULL -2 1969-12-31 16:00:00.381 -2.0 NULL 2 -2 0 -2.0 -0.0 2.0 -159.106 0.0 2.0 0.0 -10.175 -2.0 13.14 -2.0 0.0 -2 -true 55 1969-12-31 16:00:15.678 55.0 cvLH6Eat2yFsyy7p -55 55 0 55.0 0.0 -55.0 4375.415 0.0 -55.0 0.0 -10.175 55.0 -0.47781818181818186 55.0 0.0 55 -NULL -59 1969-12-31 16:00:09.498 -59.0 NULL 59 -59 0 -59.0 -0.0 59.0 -4693.6269999999995 0.0 59.0 0.0 -10.175 -59.0 0.44542372881355935 -59.0 0.0 -59 -true 28 1969-12-31 16:00:11.35 28.0 821UdmGbkEf4j -28 28 0 28.0 0.0 -28.0 2227.484 0.0 -28.0 0.0 -10.175 28.0 -0.9385714285714286 28.0 0.0 28 -true -43 1969-12-31 16:00:15.006 -43.0 821UdmGbkEf4j 43 -43 0 -43.0 -0.0 43.0 -3420.779 0.0 43.0 0.0 -10.175 -43.0 0.6111627906976744 -43.0 0.0 -43 -NULL 40 1969-12-31 16:00:08.06 40.0 NULL -40 40 0 40.0 0.0 -40.0 3182.12 0.0 -40.0 0.0 -10.175 40.0 -0.657 40.0 0.0 40 -true 24 1969-12-31 16:00:07.164 24.0 1cGVWH7n1QU -24 24 0 24.0 0.0 -24.0 1909.272 0.0 -24.0 0.0 -10.175 24.0 -1.095 24.0 0.0 24 -NULL 55 1969-12-31 16:00:09.69 55.0 NULL -55 55 0 55.0 0.0 -55.0 4375.415 0.0 -55.0 0.0 -10.175 55.0 -0.47781818181818186 55.0 0.0 55 -NULL 51 1969-12-31 16:00:08.063 51.0 NULL -51 51 0 51.0 0.0 -51.0 4057.203 0.0 -51.0 0.0 -10.175 51.0 -0.5152941176470588 51.0 0.0 51 -NULL -17 1969-12-31 16:00:09.761 -17.0 NULL 17 -17 0 -17.0 -0.0 17.0 -1352.4009999999998 0.0 17.0 0.0 -10.175 -17.0 1.5458823529411765 -17.0 0.0 -17 -NULL -3 1969-12-31 16:00:11.847 -3.0 NULL 3 -3 0 -3.0 -0.0 3.0 -238.659 0.0 3.0 0.0 -10.175 -3.0 8.76 -3.0 0.0 -3 -NULL 40 1969-12-31 16:00:05.031 40.0 NULL -40 40 0 40.0 0.0 -40.0 3182.12 0.0 -40.0 0.0 -10.175 40.0 -0.657 40.0 0.0 40 -NULL 49 1969-12-31 16:00:12.006 49.0 NULL -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 -NULL 2 1969-12-31 16:00:10.045 2.0 NULL -2 2 0 2.0 0.0 -2.0 159.106 0.0 -2.0 0.0 -10.175 2.0 -13.14 2.0 0.0 2 -NULL -11 1969-12-31 16:00:07.184 -11.0 NULL 11 -11 0 -11.0 -0.0 11.0 -875.083 0.0 11.0 0.0 -10.175 -11.0 2.389090909090909 -11.0 0.0 -11 -true -20 1969-12-31 16:00:10.077 -20.0 821UdmGbkEf4j 20 -20 0 -20.0 -0.0 20.0 -1591.06 0.0 20.0 0.0 -10.175 -20.0 1.314 -20.0 0.0 -20 -NULL -50 1969-12-31 16:00:02.434 -50.0 NULL 50 -50 0 -50.0 -0.0 50.0 -3977.6499999999996 0.0 50.0 0.0 -10.175 -50.0 0.5256000000000001 -50.0 0.0 -50 -true 61 1969-12-31 16:00:05.391 61.0 1cGVWH7n1QU -61 61 0 61.0 0.0 -61.0 4852.733 0.0 -61.0 0.0 -10.175 61.0 -0.4308196721311476 61.0 0.0 61 -true 49 1969-12-31 16:00:06.048 49.0 cvLH6Eat2yFsyy7p -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 -true 42 1969-12-31 16:00:06.511 42.0 821UdmGbkEf4j -42 42 0 42.0 0.0 -42.0 3341.2259999999997 0.0 -42.0 0.0 -10.175 42.0 -0.6257142857142858 42.0 0.0 42 -true 24 1969-12-31 16:00:07.534 24.0 1cGVWH7n1QU -24 24 0 24.0 0.0 -24.0 1909.272 0.0 -24.0 0.0 -10.175 24.0 -1.095 24.0 0.0 24 -true 47 1969-12-31 16:00:11.046 47.0 1cGVWH7n1QU -47 47 0 47.0 0.0 -47.0 3738.991 0.0 -47.0 0.0 -10.175 47.0 -0.5591489361702128 47.0 0.0 47 -NULL 56 1969-12-31 16:00:00.446 56.0 NULL -56 56 0 56.0 0.0 -56.0 4454.968 0.0 -56.0 0.0 -10.175 56.0 -0.4692857142857143 56.0 0.0 56 -true -57 1969-12-31 16:00:03.817 -57.0 821UdmGbkEf4j 57 -57 0 -57.0 -0.0 57.0 -4534.521 0.0 57.0 0.0 -10.175 -57.0 0.4610526315789474 -57.0 0.0 -57 -true -52 1969-12-31 16:00:10.304 -52.0 cvLH6Eat2yFsyy7p 52 -52 0 -52.0 -0.0 52.0 -4136.755999999999 0.0 52.0 0.0 -10.175 -52.0 0.5053846153846154 -52.0 0.0 -52 -NULL 31 1969-12-31 16:00:12.63 31.0 NULL -31 31 0 31.0 0.0 -31.0 2466.143 0.0 -31.0 0.0 -10.175 31.0 -0.847741935483871 31.0 0.0 31 -true 10 1969-12-31 16:00:16.281 10.0 1cGVWH7n1QU -10 10 0 10.0 0.0 -10.0 795.53 0.0 -10.0 0.0 -10.175 10.0 -2.628 10.0 0.0 10 -NULL 17 1969-12-31 16:00:15.338 17.0 NULL -17 17 0 17.0 0.0 -17.0 1352.4009999999998 0.0 -17.0 0.0 -10.175 17.0 -1.5458823529411765 17.0 0.0 17 -NULL 40 1969-12-31 16:00:03.406 40.0 NULL -40 40 0 40.0 0.0 -40.0 3182.12 0.0 -40.0 0.0 -10.175 40.0 -0.657 40.0 0.0 40 -NULL 42 1969-12-31 16:00:00.992 42.0 NULL -42 42 0 42.0 0.0 -42.0 3341.2259999999997 0.0 -42.0 0.0 -10.175 42.0 -0.6257142857142858 42.0 0.0 42 -true -59 1969-12-31 16:00:00.418 -59.0 cvLH6Eat2yFsyy7p 59 -59 0 -59.0 -0.0 59.0 -4693.6269999999995 0.0 59.0 0.0 -10.175 -59.0 0.44542372881355935 -59.0 0.0 -59 -true 25 1969-12-31 16:00:06.136 25.0 1cGVWH7n1QU -25 25 0 25.0 0.0 -25.0 1988.8249999999998 0.0 -25.0 0.0 -10.175 25.0 -1.0512000000000001 25.0 0.0 25 -NULL 20 1969-12-31 16:00:06.034 20.0 NULL -20 20 0 20.0 0.0 -20.0 1591.06 0.0 -20.0 0.0 -10.175 20.0 -1.314 20.0 0.0 20 -true 2 1969-12-31 16:00:01.923 2.0 cvLH6Eat2yFsyy7p -2 2 0 2.0 0.0 -2.0 159.106 0.0 -2.0 0.0 -10.175 2.0 -13.14 2.0 0.0 2 -true -2 1969-12-31 16:00:15.921 -2.0 cvLH6Eat2yFsyy7p 2 -2 0 -2.0 -0.0 2.0 -159.106 0.0 2.0 0.0 -10.175 -2.0 13.14 -2.0 0.0 -2 -NULL -50 1969-12-31 16:00:12.292 -50.0 NULL 50 -50 0 -50.0 -0.0 50.0 -3977.6499999999996 0.0 50.0 0.0 -10.175 -50.0 0.5256000000000001 -50.0 0.0 -50 -true 2 1969-12-31 16:00:14.787 2.0 1cGVWH7n1QU -2 2 0 2.0 0.0 -2.0 159.106 0.0 -2.0 0.0 -10.175 2.0 -13.14 2.0 0.0 2 -NULL -32 1969-12-31 16:00:00.527 -32.0 NULL 32 -32 0 -32.0 -0.0 32.0 -2545.696 0.0 32.0 0.0 -10.175 -32.0 0.82125 -32.0 0.0 -32 -NULL 19 1969-12-31 16:00:03.44 19.0 NULL -19 19 0 19.0 0.0 -19.0 1511.507 0.0 -19.0 0.0 -10.175 19.0 -1.3831578947368421 19.0 0.0 19 -NULL -29 1969-12-31 16:00:02.82 -29.0 NULL 29 -29 0 -29.0 -0.0 29.0 -2307.037 0.0 29.0 0.0 -10.175 -29.0 0.9062068965517242 -29.0 0.0 -29 -true -7 1969-12-31 16:00:05.662 -7.0 1cGVWH7n1QU 7 -7 0 -7.0 -0.0 7.0 -556.871 0.0 7.0 0.0 -10.175 -7.0 3.7542857142857144 -7.0 0.0 -7 -true -18 1969-12-31 16:00:13.748 -18.0 cvLH6Eat2yFsyy7p 18 -18 0 -18.0 -0.0 18.0 -1431.954 0.0 18.0 0.0 -10.175 -18.0 1.46 -18.0 0.0 -18 -true 58 1969-12-31 16:00:03.855 58.0 1cGVWH7n1QU -58 58 0 58.0 0.0 -58.0 4614.074 0.0 -58.0 0.0 -10.175 58.0 -0.4531034482758621 58.0 0.0 58 -true 58 1969-12-31 16:00:08.031 58.0 821UdmGbkEf4j -58 58 0 58.0 0.0 -58.0 4614.074 0.0 -58.0 0.0 -10.175 58.0 -0.4531034482758621 58.0 0.0 58 -NULL -42 1969-12-31 16:00:02.754 -42.0 NULL 42 -42 0 -42.0 -0.0 42.0 -3341.2259999999997 0.0 42.0 0.0 -10.175 -42.0 0.6257142857142858 -42.0 0.0 -42 -NULL -55 1969-12-31 16:00:01.138 -55.0 NULL 55 -55 0 -55.0 -0.0 55.0 -4375.415 0.0 55.0 0.0 -10.175 -55.0 0.47781818181818186 -55.0 0.0 -55 -NULL 10 1969-12-31 16:00:05.383 10.0 NULL -10 10 0 10.0 0.0 -10.0 795.53 0.0 -10.0 0.0 -10.175 10.0 -2.628 10.0 0.0 10 -NULL -24 1969-12-31 16:00:06.568 -24.0 NULL 24 -24 0 -24.0 -0.0 24.0 -1909.272 0.0 24.0 0.0 -10.175 -24.0 1.095 -24.0 0.0 -24 -true -14 1969-12-31 16:00:13.802 -14.0 cvLH6Eat2yFsyy7p 14 -14 0 -14.0 -0.0 14.0 -1113.742 0.0 14.0 0.0 -10.175 -14.0 1.8771428571428572 -14.0 0.0 -14 -true -29 1969-12-31 16:00:02.452 -29.0 cvLH6Eat2yFsyy7p 29 -29 0 -29.0 -0.0 29.0 -2307.037 0.0 29.0 0.0 -10.175 -29.0 0.9062068965517242 -29.0 0.0 -29 -true -59 1969-12-31 16:00:16.069 -59.0 cvLH6Eat2yFsyy7p 59 -59 0 -59.0 -0.0 59.0 -4693.6269999999995 0.0 59.0 0.0 -10.175 -59.0 0.44542372881355935 -59.0 0.0 -59 -NULL -35 1969-12-31 16:00:13.954 -35.0 NULL 35 -35 0 -35.0 -0.0 35.0 -2784.355 0.0 35.0 0.0 -10.175 -35.0 0.7508571428571429 -35.0 0.0 -35 -NULL 21 1969-12-31 16:00:10.652 21.0 NULL -21 21 0 21.0 0.0 -21.0 1670.6129999999998 0.0 -21.0 0.0 -10.175 21.0 -1.2514285714285716 21.0 0.0 21 -true -52 1969-12-31 16:00:14.404 -52.0 cvLH6Eat2yFsyy7p 52 -52 0 -52.0 -0.0 52.0 -4136.755999999999 0.0 52.0 0.0 -10.175 -52.0 0.5053846153846154 -52.0 0.0 -52 -NULL 8 1969-12-31 16:00:09.961 8.0 NULL -8 8 0 8.0 0.0 -8.0 636.424 0.0 -8.0 0.0 -10.175 8.0 -3.285 8.0 0.0 8 -true -58 1969-12-31 16:00:01.246 -58.0 cvLH6Eat2yFsyy7p 58 -58 0 -58.0 -0.0 58.0 -4614.074 0.0 58.0 0.0 -10.175 -58.0 0.4531034482758621 -58.0 0.0 -58 -true 6 1969-12-31 16:00:09.857 6.0 cvLH6Eat2yFsyy7p -6 6 0 6.0 0.0 -6.0 477.318 0.0 -6.0 0.0 -10.175 6.0 -4.38 6.0 0.0 6 -NULL 38 1969-12-31 16:00:13.506 38.0 NULL -38 38 0 38.0 0.0 -38.0 3023.014 0.0 -38.0 0.0 -10.175 38.0 -0.6915789473684211 38.0 0.0 38 -true -43 1969-12-31 16:00:08.384 -43.0 cvLH6Eat2yFsyy7p 43 -43 0 -43.0 -0.0 43.0 -3420.779 0.0 43.0 0.0 -10.175 -43.0 0.6111627906976744 -43.0 0.0 -43 -true 54 1969-12-31 16:00:11.256 54.0 1cGVWH7n1QU -54 54 0 54.0 0.0 -54.0 4295.862 0.0 -54.0 0.0 -10.175 54.0 -0.4866666666666667 54.0 0.0 54 -true -61 1969-12-31 16:00:03.742 -61.0 1cGVWH7n1QU 61 -61 0 -61.0 -0.0 61.0 -4852.733 0.0 61.0 0.0 -10.175 -61.0 0.4308196721311476 -61.0 0.0 -61 -true 32 1969-12-31 16:00:04.264 32.0 cvLH6Eat2yFsyy7p -32 32 0 32.0 0.0 -32.0 2545.696 0.0 -32.0 0.0 -10.175 32.0 -0.82125 32.0 0.0 32 -NULL -52 1969-12-31 16:00:02.621 -52.0 NULL 52 -52 0 -52.0 -0.0 52.0 -4136.755999999999 0.0 52.0 0.0 -10.175 -52.0 0.5053846153846154 -52.0 0.0 -52 -true 37 1969-12-31 16:00:14.889 37.0 821UdmGbkEf4j -37 37 0 37.0 0.0 -37.0 2943.461 0.0 -37.0 0.0 -10.175 37.0 -0.7102702702702703 37.0 0.0 37 -true 37 1969-12-31 16:00:06.547 37.0 1cGVWH7n1QU -37 37 0 37.0 0.0 -37.0 2943.461 0.0 -37.0 0.0 -10.175 37.0 -0.7102702702702703 37.0 0.0 37 -NULL -23 1969-12-31 16:00:13.835 -23.0 NULL 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -NULL -44 1969-12-31 16:00:04.442 -44.0 NULL 44 -44 0 -44.0 -0.0 44.0 -3500.332 0.0 44.0 0.0 -10.175 -44.0 0.5972727272727273 -44.0 0.0 -44 -NULL 17 1969-12-31 16:00:02.436 17.0 NULL -17 17 0 17.0 0.0 -17.0 1352.4009999999998 0.0 -17.0 0.0 -10.175 17.0 -1.5458823529411765 17.0 0.0 17 -NULL 7 1969-12-31 16:00:00.984 7.0 NULL -7 7 0 7.0 0.0 -7.0 556.871 0.0 -7.0 0.0 -10.175 7.0 -3.7542857142857144 7.0 0.0 7 -NULL 37 1969-12-31 16:00:01.979 37.0 NULL -37 37 0 37.0 0.0 -37.0 2943.461 0.0 -37.0 0.0 -10.175 37.0 -0.7102702702702703 37.0 0.0 37 -true -59 1969-12-31 16:00:10.532 -59.0 821UdmGbkEf4j 59 -59 0 -59.0 -0.0 59.0 -4693.6269999999995 0.0 59.0 0.0 -10.175 -59.0 0.44542372881355935 -59.0 0.0 -59 -true -24 1969-12-31 16:00:08.076 -24.0 821UdmGbkEf4j 24 -24 0 -24.0 -0.0 24.0 -1909.272 0.0 24.0 0.0 -10.175 -24.0 1.095 -24.0 0.0 -24 -true 36 1969-12-31 16:00:02.123 36.0 821UdmGbkEf4j -36 36 0 36.0 0.0 -36.0 2863.908 0.0 -36.0 0.0 -10.175 36.0 -0.73 36.0 0.0 36 -NULL -59 1969-12-31 16:00:09.401 -59.0 NULL 59 -59 0 -59.0 -0.0 59.0 -4693.6269999999995 0.0 59.0 0.0 -10.175 -59.0 0.44542372881355935 -59.0 0.0 -59 -NULL -6 1969-12-31 16:00:00.863 -6.0 NULL 6 -6 0 -6.0 -0.0 6.0 -477.318 0.0 6.0 0.0 -10.175 -6.0 4.38 -6.0 0.0 -6 -true -48 1969-12-31 16:00:12.531 -48.0 cvLH6Eat2yFsyy7p 48 -48 0 -48.0 -0.0 48.0 -3818.544 0.0 48.0 0.0 -10.175 -48.0 0.5475 -48.0 0.0 -48 -NULL 18 1969-12-31 16:00:15.89 18.0 NULL -18 18 0 18.0 0.0 -18.0 1431.954 0.0 -18.0 0.0 -10.175 18.0 -1.46 18.0 0.0 18 -true -22 1969-12-31 16:00:16.27 -22.0 821UdmGbkEf4j 22 -22 0 -22.0 -0.0 22.0 -1750.166 0.0 22.0 0.0 -10.175 -22.0 1.1945454545454546 -22.0 0.0 -22 -NULL -26 1969-12-31 16:00:03.874 -26.0 NULL 26 -26 0 -26.0 -0.0 26.0 -2068.3779999999997 0.0 26.0 0.0 -10.175 -26.0 1.0107692307692309 -26.0 0.0 -26 -NULL -19 1969-12-31 16:00:03.381 -19.0 NULL 19 -19 0 -19.0 -0.0 19.0 -1511.507 0.0 19.0 0.0 -10.175 -19.0 1.3831578947368421 -19.0 0.0 -19 -true 45 1969-12-31 16:00:05.536 45.0 1cGVWH7n1QU -45 45 0 45.0 0.0 -45.0 3579.8849999999998 0.0 -45.0 0.0 -10.175 45.0 -0.5840000000000001 45.0 0.0 45 -NULL 38 1969-12-31 16:00:05.478 38.0 NULL -38 38 0 38.0 0.0 -38.0 3023.014 0.0 -38.0 0.0 -10.175 38.0 -0.6915789473684211 38.0 0.0 38 -true 44 1969-12-31 16:00:12.55 44.0 cvLH6Eat2yFsyy7p -44 44 0 44.0 0.0 -44.0 3500.332 0.0 -44.0 0.0 -10.175 44.0 -0.5972727272727273 44.0 0.0 44 -true -55 1969-12-31 16:00:00.557 -55.0 821UdmGbkEf4j 55 -55 0 -55.0 -0.0 55.0 -4375.415 0.0 55.0 0.0 -10.175 -55.0 0.47781818181818186 -55.0 0.0 -55 -true 47 1969-12-31 16:00:02.024 47.0 1cGVWH7n1QU -47 47 0 47.0 0.0 -47.0 3738.991 0.0 -47.0 0.0 -10.175 47.0 -0.5591489361702128 47.0 0.0 47 -true 57 1969-12-31 16:00:14.889 57.0 1cGVWH7n1QU -57 57 0 57.0 0.0 -57.0 4534.521 0.0 -57.0 0.0 -10.175 57.0 -0.4610526315789474 57.0 0.0 57 -true 18 1969-12-31 16:00:01.154 18.0 1cGVWH7n1QU -18 18 0 18.0 0.0 -18.0 1431.954 0.0 -18.0 0.0 -10.175 18.0 -1.46 18.0 0.0 18 -NULL -45 1969-12-31 16:00:00.46 -45.0 NULL 45 -45 0 -45.0 -0.0 45.0 -3579.8849999999998 0.0 45.0 0.0 -10.175 -45.0 0.5840000000000001 -45.0 0.0 -45 -NULL -6 1969-12-31 16:00:06.692 -6.0 NULL 6 -6 0 -6.0 -0.0 6.0 -477.318 0.0 6.0 0.0 -10.175 -6.0 4.38 -6.0 0.0 -6 -true 5 1969-12-31 16:00:11.561 5.0 821UdmGbkEf4j -5 5 0 5.0 0.0 -5.0 397.765 0.0 -5.0 0.0 -10.175 5.0 -5.256 5.0 0.0 5 -NULL -60 1969-12-31 16:00:11.504 -60.0 NULL 60 -60 0 -60.0 -0.0 60.0 -4773.18 0.0 60.0 0.0 -10.175 -60.0 0.438 -60.0 0.0 -60 -NULL 42 1969-12-31 16:00:13.385 42.0 NULL -42 42 0 42.0 0.0 -42.0 3341.2259999999997 0.0 -42.0 0.0 -10.175 42.0 -0.6257142857142858 42.0 0.0 42 -true 16 1969-12-31 16:00:00.147 16.0 1cGVWH7n1QU -16 16 0 16.0 0.0 -16.0 1272.848 0.0 -16.0 0.0 -10.175 16.0 -1.6425 16.0 0.0 16 -NULL 39 1969-12-31 16:00:09.365 39.0 NULL -39 39 0 39.0 0.0 -39.0 3102.567 0.0 -39.0 0.0 -10.175 39.0 -0.6738461538461539 39.0 0.0 39 -true -24 1969-12-31 16:00:16.212 -24.0 821UdmGbkEf4j 24 -24 0 -24.0 -0.0 24.0 -1909.272 0.0 24.0 0.0 -10.175 -24.0 1.095 -24.0 0.0 -24 -true 1 1969-12-31 16:00:10.149 1.0 821UdmGbkEf4j -1 1 0 1.0 0.0 -1.0 79.553 0.0 -1.0 0.0 -10.175 1.0 -26.28 1.0 0.0 1 -true -58 1969-12-31 16:00:10.922 -58.0 821UdmGbkEf4j 58 -58 0 -58.0 -0.0 58.0 -4614.074 0.0 58.0 0.0 -10.175 -58.0 0.4531034482758621 -58.0 0.0 -58 -true 19 1969-12-31 16:00:09.507 19.0 1cGVWH7n1QU -19 19 0 19.0 0.0 -19.0 1511.507 0.0 -19.0 0.0 -10.175 19.0 -1.3831578947368421 19.0 0.0 19 -true 34 1969-12-31 16:00:01.975 34.0 1cGVWH7n1QU -34 34 0 34.0 0.0 -34.0 2704.8019999999997 0.0 -34.0 0.0 -10.175 34.0 -0.7729411764705882 34.0 0.0 34 -true 40 1969-12-31 16:00:08.105 40.0 821UdmGbkEf4j -40 40 0 40.0 0.0 -40.0 3182.12 0.0 -40.0 0.0 -10.175 40.0 -0.657 40.0 0.0 40 -NULL -47 1969-12-31 16:00:04.977 -47.0 NULL 47 -47 0 -47.0 -0.0 47.0 -3738.991 0.0 47.0 0.0 -10.175 -47.0 0.5591489361702128 -47.0 0.0 -47 -NULL 42 1969-12-31 16:00:02.249 42.0 NULL -42 42 0 42.0 0.0 -42.0 3341.2259999999997 0.0 -42.0 0.0 -10.175 42.0 -0.6257142857142858 42.0 0.0 42 -NULL 39 1969-12-31 16:00:10.915 39.0 NULL -39 39 0 39.0 0.0 -39.0 3102.567 0.0 -39.0 0.0 -10.175 39.0 -0.6738461538461539 39.0 0.0 39 -NULL 30 1969-12-31 16:00:14.23 30.0 NULL -30 30 0 30.0 0.0 -30.0 2386.59 0.0 -30.0 0.0 -10.175 30.0 -0.876 30.0 0.0 30 -NULL -47 1969-12-31 16:00:01.673 -47.0 NULL 47 -47 0 -47.0 -0.0 47.0 -3738.991 0.0 47.0 0.0 -10.175 -47.0 0.5591489361702128 -47.0 0.0 -47 -true 18 1969-12-31 16:00:03.703 18.0 1cGVWH7n1QU -18 18 0 18.0 0.0 -18.0 1431.954 0.0 -18.0 0.0 -10.175 18.0 -1.46 18.0 0.0 18 -true 32 1969-12-31 16:00:10.451 32.0 821UdmGbkEf4j -32 32 0 32.0 0.0 -32.0 2545.696 0.0 -32.0 0.0 -10.175 32.0 -0.82125 32.0 0.0 32 -true -4 1969-12-31 16:00:07.66 -4.0 1cGVWH7n1QU 4 -4 0 -4.0 -0.0 4.0 -318.212 0.0 4.0 0.0 -10.175 -4.0 6.57 -4.0 0.0 -4 -true 3 1969-12-31 16:00:31.808 3.0 821UdmGbkEf4j -3 3 0 3.0 0.0 -3.0 238.659 0.0 -3.0 0.0 -10.175 3.0 -8.76 3.0 0.0 3 -true 57 1969-12-31 16:00:00.646 57.0 821UdmGbkEf4j -57 57 0 57.0 0.0 -57.0 4534.521 0.0 -57.0 0.0 -10.175 57.0 -0.4610526315789474 57.0 0.0 57 -NULL -45 1969-12-31 16:00:07.592 -45.0 NULL 45 -45 0 -45.0 -0.0 45.0 -3579.8849999999998 0.0 45.0 0.0 -10.175 -45.0 0.5840000000000001 -45.0 0.0 -45 -true 28 1969-12-31 16:00:10.666 28.0 cvLH6Eat2yFsyy7p -28 28 0 28.0 0.0 -28.0 2227.484 0.0 -28.0 0.0 -10.175 28.0 -0.9385714285714286 28.0 0.0 28 -true 34 1969-12-31 16:00:00.411 34.0 821UdmGbkEf4j -34 34 0 34.0 0.0 -34.0 2704.8019999999997 0.0 -34.0 0.0 -10.175 34.0 -0.7729411764705882 34.0 0.0 34 -NULL 34 1969-12-31 16:00:12.502 34.0 NULL -34 34 0 34.0 0.0 -34.0 2704.8019999999997 0.0 -34.0 0.0 -10.175 34.0 -0.7729411764705882 34.0 0.0 34 -NULL 8 1969-12-31 16:00:09.39 8.0 NULL -8 8 0 8.0 0.0 -8.0 636.424 0.0 -8.0 0.0 -10.175 8.0 -3.285 8.0 0.0 8 -NULL 10 1969-12-31 16:00:05.348 10.0 NULL -10 10 0 10.0 0.0 -10.0 795.53 0.0 -10.0 0.0 -10.175 10.0 -2.628 10.0 0.0 10 -true 21 1969-12-31 16:00:07.784 21.0 1cGVWH7n1QU -21 21 0 21.0 0.0 -21.0 1670.6129999999998 0.0 -21.0 0.0 -10.175 21.0 -1.2514285714285716 21.0 0.0 21 -NULL 52 1969-12-31 16:00:00.158 52.0 NULL -52 52 0 52.0 0.0 -52.0 4136.755999999999 0.0 -52.0 0.0 -10.175 52.0 -0.5053846153846154 52.0 0.0 52 -true -58 1969-12-31 16:00:00.238 -58.0 1cGVWH7n1QU 58 -58 0 -58.0 -0.0 58.0 -4614.074 0.0 58.0 0.0 -10.175 -58.0 0.4531034482758621 -58.0 0.0 -58 -NULL 16 1969-12-31 16:00:05.865 16.0 NULL -16 16 0 16.0 0.0 -16.0 1272.848 0.0 -16.0 0.0 -10.175 16.0 -1.6425 16.0 0.0 16 -NULL -19 1969-12-31 16:00:00.182 -19.0 NULL 19 -19 0 -19.0 -0.0 19.0 -1511.507 0.0 19.0 0.0 -10.175 -19.0 1.3831578947368421 -19.0 0.0 -19 -true 1 1969-12-31 16:00:03.617 1.0 821UdmGbkEf4j -1 1 0 1.0 0.0 -1.0 79.553 0.0 -1.0 0.0 -10.175 1.0 -26.28 1.0 0.0 1 -NULL 12 1969-12-31 16:00:10.331 12.0 NULL -12 12 0 12.0 0.0 -12.0 954.636 0.0 -12.0 0.0 -10.175 12.0 -2.19 12.0 0.0 12 -NULL 47 1969-12-31 16:00:07.333 47.0 NULL -47 47 0 47.0 0.0 -47.0 3738.991 0.0 -47.0 0.0 -10.175 47.0 -0.5591489361702128 47.0 0.0 47 -true 48 1969-12-31 16:00:07.205 48.0 cvLH6Eat2yFsyy7p -48 48 0 48.0 0.0 -48.0 3818.544 0.0 -48.0 0.0 -10.175 48.0 -0.5475 48.0 0.0 48 -NULL 15 1969-12-31 16:00:13.114 15.0 NULL -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -true 42 1969-12-31 16:00:00.021 42.0 1cGVWH7n1QU -42 42 0 42.0 0.0 -42.0 3341.2259999999997 0.0 -42.0 0.0 -10.175 42.0 -0.6257142857142858 42.0 0.0 42 -true 23 1969-12-31 16:00:07.993 23.0 cvLH6Eat2yFsyy7p -23 23 0 23.0 0.0 -23.0 1829.719 0.0 -23.0 0.0 -10.175 23.0 -1.142608695652174 23.0 0.0 23 -NULL 49 1969-12-31 16:00:01.128 49.0 NULL -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 -NULL -32 1969-12-31 16:00:01.531 -32.0 NULL 32 -32 0 -32.0 -0.0 32.0 -2545.696 0.0 32.0 0.0 -10.175 -32.0 0.82125 -32.0 0.0 -32 -true -7 1969-12-31 16:00:09.128 -7.0 cvLH6Eat2yFsyy7p 7 -7 0 -7.0 -0.0 7.0 -556.871 0.0 7.0 0.0 -10.175 -7.0 3.7542857142857144 -7.0 0.0 -7 -NULL -5 1969-12-31 16:00:08.583 -5.0 NULL 5 -5 0 -5.0 -0.0 5.0 -397.765 0.0 5.0 0.0 -10.175 -5.0 5.256 -5.0 0.0 -5 -NULL 62 1969-12-31 16:00:10.51 62.0 NULL -62 62 0 62.0 0.0 -62.0 4932.286 0.0 -62.0 0.0 -10.175 62.0 -0.4238709677419355 62.0 0.0 62 -NULL -58 1969-12-31 16:00:12.948 -58.0 NULL 58 -58 0 -58.0 -0.0 58.0 -4614.074 0.0 58.0 0.0 -10.175 -58.0 0.4531034482758621 -58.0 0.0 -58 -NULL 26 1969-12-31 16:00:05.327 26.0 NULL -26 26 0 26.0 0.0 -26.0 2068.3779999999997 0.0 -26.0 0.0 -10.175 26.0 -1.0107692307692309 26.0 0.0 26 -NULL 47 1969-12-31 16:00:11.165 47.0 NULL -47 47 0 47.0 0.0 -47.0 3738.991 0.0 -47.0 0.0 -10.175 47.0 -0.5591489361702128 47.0 0.0 47 -NULL 4 1969-12-31 16:00:00.031 4.0 NULL -4 4 0 4.0 0.0 -4.0 318.212 0.0 -4.0 0.0 -10.175 4.0 -6.57 4.0 0.0 4 -NULL 10 1969-12-31 16:00:04.687 10.0 NULL -10 10 0 10.0 0.0 -10.0 795.53 0.0 -10.0 0.0 -10.175 10.0 -2.628 10.0 0.0 10 -NULL -49 1969-12-31 16:00:14.208 -49.0 NULL 49 -49 0 -49.0 -0.0 49.0 -3898.0969999999998 0.0 49.0 0.0 -10.175 -49.0 0.5363265306122449 -49.0 0.0 -49 -true 45 1969-12-31 16:00:05.42 45.0 cvLH6Eat2yFsyy7p -45 45 0 45.0 0.0 -45.0 3579.8849999999998 0.0 -45.0 0.0 -10.175 45.0 -0.5840000000000001 45.0 0.0 45 -true -19 1969-12-31 16:00:00.075 -19.0 cvLH6Eat2yFsyy7p 19 -19 0 -19.0 -0.0 19.0 -1511.507 0.0 19.0 0.0 -10.175 -19.0 1.3831578947368421 -19.0 0.0 -19 -NULL 58 1969-12-31 16:00:11.353 58.0 NULL -58 58 0 58.0 0.0 -58.0 4614.074 0.0 -58.0 0.0 -10.175 58.0 -0.4531034482758621 58.0 0.0 58 -true 8 1969-12-31 16:00:08.749 8.0 1cGVWH7n1QU -8 8 0 8.0 0.0 -8.0 636.424 0.0 -8.0 0.0 -10.175 8.0 -3.285 8.0 0.0 8 -NULL 31 1969-12-31 16:00:09.393 31.0 NULL -31 31 0 31.0 0.0 -31.0 2466.143 0.0 -31.0 0.0 -10.175 31.0 -0.847741935483871 31.0 0.0 31 -true -4 1969-12-31 16:00:01.567 -4.0 cvLH6Eat2yFsyy7p 4 -4 0 -4.0 -0.0 4.0 -318.212 0.0 4.0 0.0 -10.175 -4.0 6.57 -4.0 0.0 -4 -true -34 1969-12-31 16:00:06.737 -34.0 1cGVWH7n1QU 34 -34 0 -34.0 -0.0 34.0 -2704.8019999999997 0.0 34.0 0.0 -10.175 -34.0 0.7729411764705882 -34.0 0.0 -34 -NULL -20 1969-12-31 16:00:02.91 -20.0 NULL 20 -20 0 -20.0 -0.0 20.0 -1591.06 0.0 20.0 0.0 -10.175 -20.0 1.314 -20.0 0.0 -20 -NULL 0 1969-12-31 16:00:04.662 0.0 NULL 0 0 0 0.0 0.0 -0.0 0.0 0.0 -0.0 0.0 -10.175 0.0 NULL 0.0 NULL 0 -true 49 1969-12-31 16:00:12.87 49.0 821UdmGbkEf4j -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 -true 27 1969-12-31 16:00:00.18 27.0 821UdmGbkEf4j -27 27 0 27.0 0.0 -27.0 2147.931 0.0 -27.0 0.0 -10.175 27.0 -0.9733333333333334 27.0 0.0 27 -NULL -35 1969-12-31 16:00:14.819 -35.0 NULL 35 -35 0 -35.0 -0.0 35.0 -2784.355 0.0 35.0 0.0 -10.175 -35.0 0.7508571428571429 -35.0 0.0 -35 -NULL 59 1969-12-31 16:00:03.671 59.0 NULL -59 59 0 59.0 0.0 -59.0 4693.6269999999995 0.0 -59.0 0.0 -10.175 59.0 -0.44542372881355935 59.0 0.0 59 -true 48 1969-12-31 16:00:01.307 48.0 821UdmGbkEf4j -48 48 0 48.0 0.0 -48.0 3818.544 0.0 -48.0 0.0 -10.175 48.0 -0.5475 48.0 0.0 48 -true 62 1969-12-31 16:00:06.409 62.0 1cGVWH7n1QU -62 62 0 62.0 0.0 -62.0 4932.286 0.0 -62.0 0.0 -10.175 62.0 -0.4238709677419355 62.0 0.0 62 -true -63 1969-12-31 16:00:15.827 -63.0 cvLH6Eat2yFsyy7p 63 -63 0 -63.0 -0.0 63.0 -5011.839 0.0 63.0 0.0 -10.175 -63.0 0.41714285714285715 -63.0 0.0 -63 -true 21 1969-12-31 16:00:10.871 21.0 cvLH6Eat2yFsyy7p -21 21 0 21.0 0.0 -21.0 1670.6129999999998 0.0 -21.0 0.0 -10.175 21.0 -1.2514285714285716 21.0 0.0 21 -true -38 1969-12-31 16:00:01.464 -38.0 821UdmGbkEf4j 38 -38 0 -38.0 -0.0 38.0 -3023.014 0.0 38.0 0.0 -10.175 -38.0 0.6915789473684211 -38.0 0.0 -38 -NULL -59 1969-12-31 16:00:05.932 -59.0 NULL 59 -59 0 -59.0 -0.0 59.0 -4693.6269999999995 0.0 59.0 0.0 -10.175 -59.0 0.44542372881355935 -59.0 0.0 -59 -NULL 16 1969-12-31 16:00:02.208 16.0 NULL -16 16 0 16.0 0.0 -16.0 1272.848 0.0 -16.0 0.0 -10.175 16.0 -1.6425 16.0 0.0 16 -true 43 1969-12-31 16:00:10.637 43.0 cvLH6Eat2yFsyy7p -43 43 0 43.0 0.0 -43.0 3420.779 0.0 -43.0 0.0 -10.175 43.0 -0.6111627906976744 43.0 0.0 43 -true -28 1969-12-31 16:00:01.66 -28.0 cvLH6Eat2yFsyy7p 28 -28 0 -28.0 -0.0 28.0 -2227.484 0.0 28.0 0.0 -10.175 -28.0 0.9385714285714286 -28.0 0.0 -28 -true -58 1969-12-31 16:00:03.03 -58.0 821UdmGbkEf4j 58 -58 0 -58.0 -0.0 58.0 -4614.074 0.0 58.0 0.0 -10.175 -58.0 0.4531034482758621 -58.0 0.0 -58 -NULL -47 1969-12-31 16:00:08.884 -47.0 NULL 47 -47 0 -47.0 -0.0 47.0 -3738.991 0.0 47.0 0.0 -10.175 -47.0 0.5591489361702128 -47.0 0.0 -47 -true 20 1969-12-31 16:00:02.164 20.0 cvLH6Eat2yFsyy7p -20 20 0 20.0 0.0 -20.0 1591.06 0.0 -20.0 0.0 -10.175 20.0 -1.314 20.0 0.0 20 -true -36 1969-12-31 16:00:13.164 -36.0 821UdmGbkEf4j 36 -36 0 -36.0 -0.0 36.0 -2863.908 0.0 36.0 0.0 -10.175 -36.0 0.73 -36.0 0.0 -36 -true 16 1969-12-31 16:00:05.013 16.0 1cGVWH7n1QU -16 16 0 16.0 0.0 -16.0 1272.848 0.0 -16.0 0.0 -10.175 16.0 -1.6425 16.0 0.0 16 -true -52 1969-12-31 16:00:14.251 -52.0 1cGVWH7n1QU 52 -52 0 -52.0 -0.0 52.0 -4136.755999999999 0.0 52.0 0.0 -10.175 -52.0 0.5053846153846154 -52.0 0.0 -52 -NULL -4 1969-12-31 16:00:02.16 -4.0 NULL 4 -4 0 -4.0 -0.0 4.0 -318.212 0.0 4.0 0.0 -10.175 -4.0 6.57 -4.0 0.0 -4 -NULL -55 1969-12-31 16:00:13.249 -55.0 NULL 55 -55 0 -55.0 -0.0 55.0 -4375.415 0.0 55.0 0.0 -10.175 -55.0 0.47781818181818186 -55.0 0.0 -55 -NULL 38 1969-12-31 16:00:11.102 38.0 NULL -38 38 0 38.0 0.0 -38.0 3023.014 0.0 -38.0 0.0 -10.175 38.0 -0.6915789473684211 38.0 0.0 38 -true -35 1969-12-31 16:00:04.568 -35.0 1cGVWH7n1QU 35 -35 0 -35.0 -0.0 35.0 -2784.355 0.0 35.0 0.0 -10.175 -35.0 0.7508571428571429 -35.0 0.0 -35 -NULL -48 1969-12-31 16:00:00.953 -48.0 NULL 48 -48 0 -48.0 -0.0 48.0 -3818.544 0.0 48.0 0.0 -10.175 -48.0 0.5475 -48.0 0.0 -48 -NULL -62 1969-12-31 16:00:01.515 -62.0 NULL 62 -62 0 -62.0 -0.0 62.0 -4932.286 0.0 62.0 0.0 -10.175 -62.0 0.4238709677419355 -62.0 0.0 -62 -NULL 33 1969-12-31 16:00:14.872 33.0 NULL -33 33 0 33.0 0.0 -33.0 2625.249 0.0 -33.0 0.0 -10.175 33.0 -0.7963636363636364 33.0 0.0 33 -true 44 1969-12-31 16:00:05.278 44.0 1cGVWH7n1QU -44 44 0 44.0 0.0 -44.0 3500.332 0.0 -44.0 0.0 -10.175 44.0 -0.5972727272727273 44.0 0.0 44 -true 46 1969-12-31 16:00:15.456 46.0 821UdmGbkEf4j -46 46 0 46.0 0.0 -46.0 3659.438 0.0 -46.0 0.0 -10.175 46.0 -0.571304347826087 46.0 0.0 46 -true 43 1969-12-31 16:00:02.649 43.0 1cGVWH7n1QU -43 43 0 43.0 0.0 -43.0 3420.779 0.0 -43.0 0.0 -10.175 43.0 -0.6111627906976744 43.0 0.0 43 -NULL -13 1969-12-31 16:00:02.492 -13.0 NULL 13 -13 0 -13.0 -0.0 13.0 -1034.1889999999999 0.0 13.0 0.0 -10.175 -13.0 2.0215384615384617 -13.0 0.0 -13 -NULL -26 1969-12-31 16:00:07.919 -26.0 NULL 26 -26 0 -26.0 -0.0 26.0 -2068.3779999999997 0.0 26.0 0.0 -10.175 -26.0 1.0107692307692309 -26.0 0.0 -26 -true -62 1969-12-31 16:00:07.756 -62.0 1cGVWH7n1QU 62 -62 0 -62.0 -0.0 62.0 -4932.286 0.0 62.0 0.0 -10.175 -62.0 0.4238709677419355 -62.0 0.0 -62 -true 22 1969-12-31 16:00:08.487 22.0 cvLH6Eat2yFsyy7p -22 22 0 22.0 0.0 -22.0 1750.166 0.0 -22.0 0.0 -10.175 22.0 -1.1945454545454546 22.0 0.0 22 -true 21 1969-12-31 16:00:10.044 21.0 cvLH6Eat2yFsyy7p -21 21 0 21.0 0.0 -21.0 1670.6129999999998 0.0 -21.0 0.0 -10.175 21.0 -1.2514285714285716 21.0 0.0 21 -NULL 50 1969-12-31 16:00:15.232 50.0 NULL -50 50 0 50.0 0.0 -50.0 3977.6499999999996 0.0 -50.0 0.0 -10.175 50.0 -0.5256000000000001 50.0 0.0 50 -NULL -47 1969-12-31 16:00:01.05 -47.0 NULL 47 -47 0 -47.0 -0.0 47.0 -3738.991 0.0 47.0 0.0 -10.175 -47.0 0.5591489361702128 -47.0 0.0 -47 -true 21 1969-12-31 16:00:12.594 21.0 1cGVWH7n1QU -21 21 0 21.0 0.0 -21.0 1670.6129999999998 0.0 -21.0 0.0 -10.175 21.0 -1.2514285714285716 21.0 0.0 21 -true 17 1969-12-31 16:00:09.719 17.0 821UdmGbkEf4j -17 17 0 17.0 0.0 -17.0 1352.4009999999998 0.0 -17.0 0.0 -10.175 17.0 -1.5458823529411765 17.0 0.0 17 -true -5 1969-12-31 16:00:11.811 -5.0 cvLH6Eat2yFsyy7p 5 -5 0 -5.0 -0.0 5.0 -397.765 0.0 5.0 0.0 -10.175 -5.0 5.256 -5.0 0.0 -5 -true 54 1969-12-31 16:00:16.368 54.0 cvLH6Eat2yFsyy7p -54 54 0 54.0 0.0 -54.0 4295.862 0.0 -54.0 0.0 -10.175 54.0 -0.4866666666666667 54.0 0.0 54 -true 58 1969-12-31 16:00:14.693 58.0 1cGVWH7n1QU -58 58 0 58.0 0.0 -58.0 4614.074 0.0 -58.0 0.0 -10.175 58.0 -0.4531034482758621 58.0 0.0 58 -true -36 1969-12-31 16:00:15.18 -36.0 cvLH6Eat2yFsyy7p 36 -36 0 -36.0 -0.0 36.0 -2863.908 0.0 36.0 0.0 -10.175 -36.0 0.73 -36.0 0.0 -36 -true 60 1969-12-31 16:00:07.642 60.0 1cGVWH7n1QU -60 60 0 60.0 0.0 -60.0 4773.18 0.0 -60.0 0.0 -10.175 60.0 -0.438 60.0 0.0 60 -true 15 1969-12-31 16:00:02.353 15.0 821UdmGbkEf4j -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -true -15 1969-12-31 16:00:12.037 -15.0 1cGVWH7n1QU 15 -15 0 -15.0 -0.0 15.0 -1193.295 0.0 15.0 0.0 -10.175 -15.0 1.752 -15.0 0.0 -15 -NULL 62 1969-12-31 16:00:13.422 62.0 NULL -62 62 0 62.0 0.0 -62.0 4932.286 0.0 -62.0 0.0 -10.175 62.0 -0.4238709677419355 62.0 0.0 62 -true -23 1969-12-31 16:00:03.701 -23.0 1cGVWH7n1QU 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -true 39 1969-12-31 16:00:02.535 39.0 1cGVWH7n1QU -39 39 0 39.0 0.0 -39.0 3102.567 0.0 -39.0 0.0 -10.175 39.0 -0.6738461538461539 39.0 0.0 39 -NULL 20 1969-12-31 16:00:08.388 20.0 NULL -20 20 0 20.0 0.0 -20.0 1591.06 0.0 -20.0 0.0 -10.175 20.0 -1.314 20.0 0.0 20 -NULL 21 1969-12-31 16:00:08.048 21.0 NULL -21 21 0 21.0 0.0 -21.0 1670.6129999999998 0.0 -21.0 0.0 -10.175 21.0 -1.2514285714285716 21.0 0.0 21 -NULL 3 1969-12-31 16:00:01.057 3.0 NULL -3 3 0 3.0 0.0 -3.0 238.659 0.0 -3.0 0.0 -10.175 3.0 -8.76 3.0 0.0 3 -NULL -31 1969-12-31 16:00:05.643 -31.0 NULL 31 -31 0 -31.0 -0.0 31.0 -2466.143 0.0 31.0 0.0 -10.175 -31.0 0.847741935483871 -31.0 0.0 -31 -NULL 48 1969-12-31 16:00:07.08 48.0 NULL -48 48 0 48.0 0.0 -48.0 3818.544 0.0 -48.0 0.0 -10.175 48.0 -0.5475 48.0 0.0 48 -true -27 1969-12-31 16:00:00.186 -27.0 1cGVWH7n1QU 27 -27 0 -27.0 -0.0 27.0 -2147.931 0.0 27.0 0.0 -10.175 -27.0 0.9733333333333334 -27.0 0.0 -27 -NULL 59 1969-12-31 16:00:00.419 59.0 NULL -59 59 0 59.0 0.0 -59.0 4693.6269999999995 0.0 -59.0 0.0 -10.175 59.0 -0.44542372881355935 59.0 0.0 59 -true 29 1969-12-31 16:00:05.536 29.0 1cGVWH7n1QU -29 29 0 29.0 0.0 -29.0 2307.037 0.0 -29.0 0.0 -10.175 29.0 -0.9062068965517242 29.0 0.0 29 -NULL 23 1969-12-31 16:00:08.307 23.0 NULL -23 23 0 23.0 0.0 -23.0 1829.719 0.0 -23.0 0.0 -10.175 23.0 -1.142608695652174 23.0 0.0 23 -true 34 1969-12-31 16:00:00.668 34.0 cvLH6Eat2yFsyy7p -34 34 0 34.0 0.0 -34.0 2704.8019999999997 0.0 -34.0 0.0 -10.175 34.0 -0.7729411764705882 34.0 0.0 34 -true -54 1969-12-31 16:00:00.023 -54.0 1cGVWH7n1QU 54 -54 0 -54.0 -0.0 54.0 -4295.862 0.0 54.0 0.0 -10.175 -54.0 0.4866666666666667 -54.0 0.0 -54 -NULL 23 1969-12-31 16:00:14.241 23.0 NULL -23 23 0 23.0 0.0 -23.0 1829.719 0.0 -23.0 0.0 -10.175 23.0 -1.142608695652174 23.0 0.0 23 -NULL 57 1969-12-31 16:00:13.023 57.0 NULL -57 57 0 57.0 0.0 -57.0 4534.521 0.0 -57.0 0.0 -10.175 57.0 -0.4610526315789474 57.0 0.0 57 -true -56 1969-12-31 16:00:00.575 -56.0 cvLH6Eat2yFsyy7p 56 -56 0 -56.0 -0.0 56.0 -4454.968 0.0 56.0 0.0 -10.175 -56.0 0.4692857142857143 -56.0 0.0 -56 -true 58 1969-12-31 16:00:07.377 58.0 1cGVWH7n1QU -58 58 0 58.0 0.0 -58.0 4614.074 0.0 -58.0 0.0 -10.175 58.0 -0.4531034482758621 58.0 0.0 58 -true 40 1969-12-31 16:00:05.521 40.0 cvLH6Eat2yFsyy7p -40 40 0 40.0 0.0 -40.0 3182.12 0.0 -40.0 0.0 -10.175 40.0 -0.657 40.0 0.0 40 -true -47 1969-12-31 16:00:11.938 -47.0 1cGVWH7n1QU 47 -47 0 -47.0 -0.0 47.0 -3738.991 0.0 47.0 0.0 -10.175 -47.0 0.5591489361702128 -47.0 0.0 -47 -NULL -53 1969-12-31 16:00:11.936 -53.0 NULL 53 -53 0 -53.0 -0.0 53.0 -4216.309 0.0 53.0 0.0 -10.175 -53.0 0.4958490566037736 -53.0 0.0 -53 -true -2 1969-12-31 16:00:13.941 -2.0 cvLH6Eat2yFsyy7p 2 -2 0 -2.0 -0.0 2.0 -159.106 0.0 2.0 0.0 -10.175 -2.0 13.14 -2.0 0.0 -2 -NULL -3 1969-12-31 16:00:01.822 -3.0 NULL 3 -3 0 -3.0 -0.0 3.0 -238.659 0.0 3.0 0.0 -10.175 -3.0 8.76 -3.0 0.0 -3 -true -12 1969-12-31 16:00:04.871 -12.0 cvLH6Eat2yFsyy7p 12 -12 0 -12.0 -0.0 12.0 -954.636 0.0 12.0 0.0 -10.175 -12.0 2.19 -12.0 0.0 -12 -NULL -48 1969-12-31 16:00:15.63 -48.0 NULL 48 -48 0 -48.0 -0.0 48.0 -3818.544 0.0 48.0 0.0 -10.175 -48.0 0.5475 -48.0 0.0 -48 -NULL -29 1969-12-31 16:00:11.703 -29.0 NULL 29 -29 0 -29.0 -0.0 29.0 -2307.037 0.0 29.0 0.0 -10.175 -29.0 0.9062068965517242 -29.0 0.0 -29 -true -28 1969-12-31 16:00:00.185 -28.0 821UdmGbkEf4j 28 -28 0 -28.0 -0.0 28.0 -2227.484 0.0 28.0 0.0 -10.175 -28.0 0.9385714285714286 -28.0 0.0 -28 -true 53 1969-12-31 16:00:06.654 53.0 821UdmGbkEf4j -53 53 0 53.0 0.0 -53.0 4216.309 0.0 -53.0 0.0 -10.175 53.0 -0.4958490566037736 53.0 0.0 53 -true -27 1969-12-31 16:00:00.094 -27.0 cvLH6Eat2yFsyy7p 27 -27 0 -27.0 -0.0 27.0 -2147.931 0.0 27.0 0.0 -10.175 -27.0 0.9733333333333334 -27.0 0.0 -27 -true -17 1969-12-31 16:00:11.767 -17.0 cvLH6Eat2yFsyy7p 17 -17 0 -17.0 -0.0 17.0 -1352.4009999999998 0.0 17.0 0.0 -10.175 -17.0 1.5458823529411765 -17.0 0.0 -17 -NULL 20 1969-12-31 16:00:06.754 20.0 NULL -20 20 0 20.0 0.0 -20.0 1591.06 0.0 -20.0 0.0 -10.175 20.0 -1.314 20.0 0.0 20 -NULL -22 1969-12-31 16:00:00.586 -22.0 NULL 22 -22 0 -22.0 -0.0 22.0 -1750.166 0.0 22.0 0.0 -10.175 -22.0 1.1945454545454546 -22.0 0.0 -22 -true 13 1969-12-31 16:00:16.045 13.0 cvLH6Eat2yFsyy7p -13 13 0 13.0 0.0 -13.0 1034.1889999999999 0.0 -13.0 0.0 -10.175 13.0 -2.0215384615384617 13.0 0.0 13 -true -19 1969-12-31 16:00:06.036 -19.0 821UdmGbkEf4j 19 -19 0 -19.0 -0.0 19.0 -1511.507 0.0 19.0 0.0 -10.175 -19.0 1.3831578947368421 -19.0 0.0 -19 -true -18 1969-12-31 16:00:15.324 -18.0 821UdmGbkEf4j 18 -18 0 -18.0 -0.0 18.0 -1431.954 0.0 18.0 0.0 -10.175 -18.0 1.46 -18.0 0.0 -18 -true 24 1969-12-31 16:00:08.868 24.0 821UdmGbkEf4j -24 24 0 24.0 0.0 -24.0 1909.272 0.0 -24.0 0.0 -10.175 24.0 -1.095 24.0 0.0 24 -true -7 1969-12-31 16:00:08.341 -7.0 821UdmGbkEf4j 7 -7 0 -7.0 -0.0 7.0 -556.871 0.0 7.0 0.0 -10.175 -7.0 3.7542857142857144 -7.0 0.0 -7 -NULL -18 1969-12-31 16:00:04.699 -18.0 NULL 18 -18 0 -18.0 -0.0 18.0 -1431.954 0.0 18.0 0.0 -10.175 -18.0 1.46 -18.0 0.0 -18 -true 10 1969-12-31 16:00:04.197 10.0 cvLH6Eat2yFsyy7p -10 10 0 10.0 0.0 -10.0 795.53 0.0 -10.0 0.0 -10.175 10.0 -2.628 10.0 0.0 10 -true -11 1969-12-31 16:00:16.232 -11.0 cvLH6Eat2yFsyy7p 11 -11 0 -11.0 -0.0 11.0 -875.083 0.0 11.0 0.0 -10.175 -11.0 2.389090909090909 -11.0 0.0 -11 -NULL -59 1969-12-31 16:00:12.008 -59.0 NULL 59 -59 0 -59.0 -0.0 59.0 -4693.6269999999995 0.0 59.0 0.0 -10.175 -59.0 0.44542372881355935 -59.0 0.0 -59 -true -35 1969-12-31 16:00:03.708 -35.0 cvLH6Eat2yFsyy7p 35 -35 0 -35.0 -0.0 35.0 -2784.355 0.0 35.0 0.0 -10.175 -35.0 0.7508571428571429 -35.0 0.0 -35 -NULL 51 1969-12-31 16:00:08.949 51.0 NULL -51 51 0 51.0 0.0 -51.0 4057.203 0.0 -51.0 0.0 -10.175 51.0 -0.5152941176470588 51.0 0.0 51 -NULL 48 1969-12-31 16:00:05.616 48.0 NULL -48 48 0 48.0 0.0 -48.0 3818.544 0.0 -48.0 0.0 -10.175 48.0 -0.5475 48.0 0.0 48 -NULL -39 1969-12-31 16:00:14.911 -39.0 NULL 39 -39 0 -39.0 -0.0 39.0 -3102.567 0.0 39.0 0.0 -10.175 -39.0 0.6738461538461539 -39.0 0.0 -39 -NULL 60 1969-12-31 16:00:08.373 60.0 NULL -60 60 0 60.0 0.0 -60.0 4773.18 0.0 -60.0 0.0 -10.175 60.0 -0.438 60.0 0.0 60 -NULL -19 1969-12-31 16:00:05.963 -19.0 NULL 19 -19 0 -19.0 -0.0 19.0 -1511.507 0.0 19.0 0.0 -10.175 -19.0 1.3831578947368421 -19.0 0.0 -19 -true 51 1969-12-31 16:00:00.383 51.0 821UdmGbkEf4j -51 51 0 51.0 0.0 -51.0 4057.203 0.0 -51.0 0.0 -10.175 51.0 -0.5152941176470588 51.0 0.0 51 -NULL -9 1969-12-31 16:00:13.071 -9.0 NULL 9 -9 0 -9.0 -0.0 9.0 -715.977 0.0 9.0 0.0 -10.175 -9.0 2.92 -9.0 0.0 -9 -true 14 1969-12-31 16:00:00.34 14.0 821UdmGbkEf4j -14 14 0 14.0 0.0 -14.0 1113.742 0.0 -14.0 0.0 -10.175 14.0 -1.8771428571428572 14.0 0.0 14 -true -25 1969-12-31 16:00:10.484 -25.0 821UdmGbkEf4j 25 -25 0 -25.0 -0.0 25.0 -1988.8249999999998 0.0 25.0 0.0 -10.175 -25.0 1.0512000000000001 -25.0 0.0 -25 -true -12 1969-12-31 16:00:06.245 -12.0 1cGVWH7n1QU 12 -12 0 -12.0 -0.0 12.0 -954.636 0.0 12.0 0.0 -10.175 -12.0 2.19 -12.0 0.0 -12 -true -50 1969-12-31 16:00:07.249 -50.0 821UdmGbkEf4j 50 -50 0 -50.0 -0.0 50.0 -3977.6499999999996 0.0 50.0 0.0 -10.175 -50.0 0.5256000000000001 -50.0 0.0 -50 -NULL -43 1969-12-31 16:00:14.536 -43.0 NULL 43 -43 0 -43.0 -0.0 43.0 -3420.779 0.0 43.0 0.0 -10.175 -43.0 0.6111627906976744 -43.0 0.0 -43 -true 46 1969-12-31 16:00:00.971 46.0 821UdmGbkEf4j -46 46 0 46.0 0.0 -46.0 3659.438 0.0 -46.0 0.0 -10.175 46.0 -0.571304347826087 46.0 0.0 46 -NULL -3 1969-12-31 16:00:11.642 -3.0 NULL 3 -3 0 -3.0 -0.0 3.0 -238.659 0.0 3.0 0.0 -10.175 -3.0 8.76 -3.0 0.0 -3 -true -42 1969-12-31 16:00:02.57 -42.0 1cGVWH7n1QU 42 -42 0 -42.0 -0.0 42.0 -3341.2259999999997 0.0 42.0 0.0 -10.175 -42.0 0.6257142857142858 -42.0 0.0 -42 -true -20 1969-12-31 16:00:02.353 -20.0 821UdmGbkEf4j 20 -20 0 -20.0 -0.0 20.0 -1591.06 0.0 20.0 0.0 -10.175 -20.0 1.314 -20.0 0.0 -20 -true -61 1969-12-31 16:00:02.497 -61.0 cvLH6Eat2yFsyy7p 61 -61 0 -61.0 -0.0 61.0 -4852.733 0.0 61.0 0.0 -10.175 -61.0 0.4308196721311476 -61.0 0.0 -61 -true 49 1969-12-31 16:00:14.778 49.0 cvLH6Eat2yFsyy7p -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 -true -50 1969-12-31 16:00:12.339 -50.0 cvLH6Eat2yFsyy7p 50 -50 0 -50.0 -0.0 50.0 -3977.6499999999996 0.0 50.0 0.0 -10.175 -50.0 0.5256000000000001 -50.0 0.0 -50 -true -34 1969-12-31 16:00:06.557 -34.0 cvLH6Eat2yFsyy7p 34 -34 0 -34.0 -0.0 34.0 -2704.8019999999997 0.0 34.0 0.0 -10.175 -34.0 0.7729411764705882 -34.0 0.0 -34 -true -25 1969-12-31 16:00:16.285 -25.0 821UdmGbkEf4j 25 -25 0 -25.0 -0.0 25.0 -1988.8249999999998 0.0 25.0 0.0 -10.175 -25.0 1.0512000000000001 -25.0 0.0 -25 -NULL -60 1969-12-31 16:00:10.548 -60.0 NULL 60 -60 0 -60.0 -0.0 60.0 -4773.18 0.0 60.0 0.0 -10.175 -60.0 0.438 -60.0 0.0 -60 -NULL -50 1969-12-31 16:00:03.09 -50.0 NULL 50 -50 0 -50.0 -0.0 50.0 -3977.6499999999996 0.0 50.0 0.0 -10.175 -50.0 0.5256000000000001 -50.0 0.0 -50 -true 10 1969-12-31 16:00:03.631 10.0 1cGVWH7n1QU -10 10 0 10.0 0.0 -10.0 795.53 0.0 -10.0 0.0 -10.175 10.0 -2.628 10.0 0.0 10 -true 27 1969-12-31 16:00:11.668 27.0 cvLH6Eat2yFsyy7p -27 27 0 27.0 0.0 -27.0 2147.931 0.0 -27.0 0.0 -10.175 27.0 -0.9733333333333334 27.0 0.0 27 -NULL -13 1969-12-31 16:00:03.11 -13.0 NULL 13 -13 0 -13.0 -0.0 13.0 -1034.1889999999999 0.0 13.0 0.0 -10.175 -13.0 2.0215384615384617 -13.0 0.0 -13 -true 31 1969-12-31 16:00:02.219 31.0 1cGVWH7n1QU -31 31 0 31.0 0.0 -31.0 2466.143 0.0 -31.0 0.0 -10.175 31.0 -0.847741935483871 31.0 0.0 31 -NULL -8 1969-12-31 16:00:00.475 -8.0 NULL 8 -8 0 -8.0 -0.0 8.0 -636.424 0.0 8.0 0.0 -10.175 -8.0 3.285 -8.0 0.0 -8 -true 9 1969-12-31 16:00:15.745 9.0 cvLH6Eat2yFsyy7p -9 9 0 9.0 0.0 -9.0 715.977 0.0 -9.0 0.0 -10.175 9.0 -2.92 9.0 0.0 9 -NULL -14 1969-12-31 16:00:14.831 -14.0 NULL 14 -14 0 -14.0 -0.0 14.0 -1113.742 0.0 14.0 0.0 -10.175 -14.0 1.8771428571428572 -14.0 0.0 -14 -NULL -41 1969-12-31 16:00:11.565 -41.0 NULL 41 -41 0 -41.0 -0.0 41.0 -3261.673 0.0 41.0 0.0 -10.175 -41.0 0.6409756097560976 -41.0 0.0 -41 -NULL -34 1969-12-31 16:00:08.233 -34.0 NULL 34 -34 0 -34.0 -0.0 34.0 -2704.8019999999997 0.0 34.0 0.0 -10.175 -34.0 0.7729411764705882 -34.0 0.0 -34 -NULL 45 1969-12-31 16:00:11.515 45.0 NULL -45 45 0 45.0 0.0 -45.0 3579.8849999999998 0.0 -45.0 0.0 -10.175 45.0 -0.5840000000000001 45.0 0.0 45 -NULL -25 1969-12-31 16:00:03.549 -25.0 NULL 25 -25 0 -25.0 -0.0 25.0 -1988.8249999999998 0.0 25.0 0.0 -10.175 -25.0 1.0512000000000001 -25.0 0.0 -25 -true 52 1969-12-31 16:00:15.483 52.0 1cGVWH7n1QU -52 52 0 52.0 0.0 -52.0 4136.755999999999 0.0 -52.0 0.0 -10.175 52.0 -0.5053846153846154 52.0 0.0 52 -true 42 1969-12-31 16:00:11.581 42.0 1cGVWH7n1QU -42 42 0 42.0 0.0 -42.0 3341.2259999999997 0.0 -42.0 0.0 -10.175 42.0 -0.6257142857142858 42.0 0.0 42 -true -27 1969-12-31 16:00:13.117 -27.0 821UdmGbkEf4j 27 -27 0 -27.0 -0.0 27.0 -2147.931 0.0 27.0 0.0 -10.175 -27.0 0.9733333333333334 -27.0 0.0 -27 -NULL 28 1969-12-31 16:00:12.334 28.0 NULL -28 28 0 28.0 0.0 -28.0 2227.484 0.0 -28.0 0.0 -10.175 28.0 -0.9385714285714286 28.0 0.0 28 -NULL -55 1969-12-31 16:00:00.547 -55.0 NULL 55 -55 0 -55.0 -0.0 55.0 -4375.415 0.0 55.0 0.0 -10.175 -55.0 0.47781818181818186 -55.0 0.0 -55 -NULL 44 1969-12-31 16:00:04.431 44.0 NULL -44 44 0 44.0 0.0 -44.0 3500.332 0.0 -44.0 0.0 -10.175 44.0 -0.5972727272727273 44.0 0.0 44 -NULL 1 1969-12-31 16:00:14.567 1.0 NULL -1 1 0 1.0 0.0 -1.0 79.553 0.0 -1.0 0.0 -10.175 1.0 -26.28 1.0 0.0 1 -true -16 1969-12-31 16:00:08.402 -16.0 cvLH6Eat2yFsyy7p 16 -16 0 -16.0 -0.0 16.0 -1272.848 0.0 16.0 0.0 -10.175 -16.0 1.6425 -16.0 0.0 -16 -NULL -48 1969-12-31 16:00:00.172 -48.0 NULL 48 -48 0 -48.0 -0.0 48.0 -3818.544 0.0 48.0 0.0 -10.175 -48.0 0.5475 -48.0 0.0 -48 -NULL 1 1969-12-31 16:00:08.488 1.0 NULL -1 1 0 1.0 0.0 -1.0 79.553 0.0 -1.0 0.0 -10.175 1.0 -26.28 1.0 0.0 1 -true -41 1969-12-31 16:00:03.81 -41.0 1cGVWH7n1QU 41 -41 0 -41.0 -0.0 41.0 -3261.673 0.0 41.0 0.0 -10.175 -41.0 0.6409756097560976 -41.0 0.0 -41 -true -18 1969-12-31 16:00:08.671 -18.0 1cGVWH7n1QU 18 -18 0 -18.0 -0.0 18.0 -1431.954 0.0 18.0 0.0 -10.175 -18.0 1.46 -18.0 0.0 -18 -true -50 1969-12-31 16:00:06.726 -50.0 1cGVWH7n1QU 50 -50 0 -50.0 -0.0 50.0 -3977.6499999999996 0.0 50.0 0.0 -10.175 -50.0 0.5256000000000001 -50.0 0.0 -50 -NULL 45 1969-12-31 16:00:15.07 45.0 NULL -45 45 0 45.0 0.0 -45.0 3579.8849999999998 0.0 -45.0 0.0 -10.175 45.0 -0.5840000000000001 45.0 0.0 45 -true -39 1969-12-31 16:00:15.263 -39.0 1cGVWH7n1QU 39 -39 0 -39.0 -0.0 39.0 -3102.567 0.0 39.0 0.0 -10.175 -39.0 0.6738461538461539 -39.0 0.0 -39 -true 49 1969-12-31 16:00:13.146 49.0 1cGVWH7n1QU -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 -true -46 1969-12-31 16:00:07.878 -46.0 1cGVWH7n1QU 46 -46 0 -46.0 -0.0 46.0 -3659.438 0.0 46.0 0.0 -10.175 -46.0 0.571304347826087 -46.0 0.0 -46 -NULL -33 1969-12-31 16:00:13.638 -33.0 NULL 33 -33 0 -33.0 -0.0 33.0 -2625.249 0.0 33.0 0.0 -10.175 -33.0 0.7963636363636364 -33.0 0.0 -33 -true -4 1969-12-31 16:00:07.238 -4.0 cvLH6Eat2yFsyy7p 4 -4 0 -4.0 -0.0 4.0 -318.212 0.0 4.0 0.0 -10.175 -4.0 6.57 -4.0 0.0 -4 -NULL 59 1969-12-31 16:00:03.463 59.0 NULL -59 59 0 59.0 0.0 -59.0 4693.6269999999995 0.0 -59.0 0.0 -10.175 59.0 -0.44542372881355935 59.0 0.0 59 -NULL -11 1969-12-31 16:00:10.496 -11.0 NULL 11 -11 0 -11.0 -0.0 11.0 -875.083 0.0 11.0 0.0 -10.175 -11.0 2.389090909090909 -11.0 0.0 -11 -NULL -11 1969-12-31 16:00:11.09 -11.0 NULL 11 -11 0 -11.0 -0.0 11.0 -875.083 0.0 11.0 0.0 -10.175 -11.0 2.389090909090909 -11.0 0.0 -11 -true -12 1969-12-31 16:00:03.985 -12.0 1cGVWH7n1QU 12 -12 0 -12.0 -0.0 12.0 -954.636 0.0 12.0 0.0 -10.175 -12.0 2.19 -12.0 0.0 -12 -NULL -2 1969-12-31 16:00:00.741 -2.0 NULL 2 -2 0 -2.0 -0.0 2.0 -159.106 0.0 2.0 0.0 -10.175 -2.0 13.14 -2.0 0.0 -2 -NULL -7 1969-12-31 16:00:16.076 -7.0 NULL 7 -7 0 -7.0 -0.0 7.0 -556.871 0.0 7.0 0.0 -10.175 -7.0 3.7542857142857144 -7.0 0.0 -7 -true -33 1969-12-31 16:00:01.399 -33.0 821UdmGbkEf4j 33 -33 0 -33.0 -0.0 33.0 -2625.249 0.0 33.0 0.0 -10.175 -33.0 0.7963636363636364 -33.0 0.0 -33 -true 54 1969-12-31 16:00:13.953 54.0 cvLH6Eat2yFsyy7p -54 54 0 54.0 0.0 -54.0 4295.862 0.0 -54.0 0.0 -10.175 54.0 -0.4866666666666667 54.0 0.0 54 -NULL -26 1969-12-31 16:00:04.435 -26.0 NULL 26 -26 0 -26.0 -0.0 26.0 -2068.3779999999997 0.0 26.0 0.0 -10.175 -26.0 1.0107692307692309 -26.0 0.0 -26 -true 42 1969-12-31 16:00:08.791 42.0 1cGVWH7n1QU -42 42 0 42.0 0.0 -42.0 3341.2259999999997 0.0 -42.0 0.0 -10.175 42.0 -0.6257142857142858 42.0 0.0 42 -NULL -38 1969-12-31 16:00:14.118 -38.0 NULL 38 -38 0 -38.0 -0.0 38.0 -3023.014 0.0 38.0 0.0 -10.175 -38.0 0.6915789473684211 -38.0 0.0 -38 -NULL 8 1969-12-31 16:00:08.757 8.0 NULL -8 8 0 8.0 0.0 -8.0 636.424 0.0 -8.0 0.0 -10.175 8.0 -3.285 8.0 0.0 8 -NULL 54 1969-12-31 16:00:06.945 54.0 NULL -54 54 0 54.0 0.0 -54.0 4295.862 0.0 -54.0 0.0 -10.175 54.0 -0.4866666666666667 54.0 0.0 54 -NULL -56 1969-12-31 16:00:13.602 -56.0 NULL 56 -56 0 -56.0 -0.0 56.0 -4454.968 0.0 56.0 0.0 -10.175 -56.0 0.4692857142857143 -56.0 0.0 -56 -true -28 1969-12-31 16:00:00.39 -28.0 821UdmGbkEf4j 28 -28 0 -28.0 -0.0 28.0 -2227.484 0.0 28.0 0.0 -10.175 -28.0 0.9385714285714286 -28.0 0.0 -28 -NULL 34 1969-12-31 16:00:14.654 34.0 NULL -34 34 0 34.0 0.0 -34.0 2704.8019999999997 0.0 -34.0 0.0 -10.175 34.0 -0.7729411764705882 34.0 0.0 34 -true -38 1969-12-31 16:00:01.135 -38.0 cvLH6Eat2yFsyy7p 38 -38 0 -38.0 -0.0 38.0 -3023.014 0.0 38.0 0.0 -10.175 -38.0 0.6915789473684211 -38.0 0.0 -38 -true 1 1969-12-31 16:00:10.401 1.0 821UdmGbkEf4j -1 1 0 1.0 0.0 -1.0 79.553 0.0 -1.0 0.0 -10.175 1.0 -26.28 1.0 0.0 1 -true -25 1969-12-31 16:00:13.947 -25.0 821UdmGbkEf4j 25 -25 0 -25.0 -0.0 25.0 -1988.8249999999998 0.0 25.0 0.0 -10.175 -25.0 1.0512000000000001 -25.0 0.0 -25 -true -49 1969-12-31 16:00:12.602 -49.0 1cGVWH7n1QU 49 -49 0 -49.0 -0.0 49.0 -3898.0969999999998 0.0 49.0 0.0 -10.175 -49.0 0.5363265306122449 -49.0 0.0 -49 -true -49 1969-12-31 16:00:11.021 -49.0 1cGVWH7n1QU 49 -49 0 -49.0 -0.0 49.0 -3898.0969999999998 0.0 49.0 0.0 -10.175 -49.0 0.5363265306122449 -49.0 0.0 -49 -true -58 1969-12-31 16:00:02.915 -58.0 821UdmGbkEf4j 58 -58 0 -58.0 -0.0 58.0 -4614.074 0.0 58.0 0.0 -10.175 -58.0 0.4531034482758621 -58.0 0.0 -58 -true 40 1969-12-31 16:00:15.973 40.0 cvLH6Eat2yFsyy7p -40 40 0 40.0 0.0 -40.0 3182.12 0.0 -40.0 0.0 -10.175 40.0 -0.657 40.0 0.0 40 -NULL 14 1969-12-31 16:00:09.21 14.0 NULL -14 14 0 14.0 0.0 -14.0 1113.742 0.0 -14.0 0.0 -10.175 14.0 -1.8771428571428572 14.0 0.0 14 -true -64 1969-12-31 16:00:03.088 -64.0 cvLH6Eat2yFsyy7p 64 -64 0 -64.0 -0.0 64.0 -5091.392 0.0 64.0 0.0 -10.175 -64.0 0.410625 -64.0 0.0 -64 -NULL -50 1969-12-31 16:00:13.253 -50.0 NULL 50 -50 0 -50.0 -0.0 50.0 -3977.6499999999996 0.0 50.0 0.0 -10.175 -50.0 0.5256000000000001 -50.0 0.0 -50 -true -13 1969-12-31 16:00:04.238 -13.0 1cGVWH7n1QU 13 -13 0 -13.0 -0.0 13.0 -1034.1889999999999 0.0 13.0 0.0 -10.175 -13.0 2.0215384615384617 -13.0 0.0 -13 -true 5 1969-12-31 16:00:16.144 5.0 1cGVWH7n1QU -5 5 0 5.0 0.0 -5.0 397.765 0.0 -5.0 0.0 -10.175 5.0 -5.256 5.0 0.0 5 -true 44 1969-12-31 16:00:04.242 44.0 1cGVWH7n1QU -44 44 0 44.0 0.0 -44.0 3500.332 0.0 -44.0 0.0 -10.175 44.0 -0.5972727272727273 44.0 0.0 44 -true 8 1969-12-31 16:00:09.373 8.0 821UdmGbkEf4j -8 8 0 8.0 0.0 -8.0 636.424 0.0 -8.0 0.0 -10.175 8.0 -3.285 8.0 0.0 8 -NULL -26 1969-12-31 16:00:14.661 -26.0 NULL 26 -26 0 -26.0 -0.0 26.0 -2068.3779999999997 0.0 26.0 0.0 -10.175 -26.0 1.0107692307692309 -26.0 0.0 -26 -NULL -42 1969-12-31 16:00:12.748 -42.0 NULL 42 -42 0 -42.0 -0.0 42.0 -3341.2259999999997 0.0 42.0 0.0 -10.175 -42.0 0.6257142857142858 -42.0 0.0 -42 -NULL 28 1969-12-31 16:00:13.409 28.0 NULL -28 28 0 28.0 0.0 -28.0 2227.484 0.0 -28.0 0.0 -10.175 28.0 -0.9385714285714286 28.0 0.0 28 -true 45 1969-12-31 16:00:05.109 45.0 cvLH6Eat2yFsyy7p -45 45 0 45.0 0.0 -45.0 3579.8849999999998 0.0 -45.0 0.0 -10.175 45.0 -0.5840000000000001 45.0 0.0 45 -true 40 1969-12-31 16:00:05.614 40.0 1cGVWH7n1QU -40 40 0 40.0 0.0 -40.0 3182.12 0.0 -40.0 0.0 -10.175 40.0 -0.657 40.0 0.0 40 -true 1 1969-12-31 16:00:06.974 1.0 cvLH6Eat2yFsyy7p -1 1 0 1.0 0.0 -1.0 79.553 0.0 -1.0 0.0 -10.175 1.0 -26.28 1.0 0.0 1 -true -9 1969-12-31 16:00:11.632 -9.0 cvLH6Eat2yFsyy7p 9 -9 0 -9.0 -0.0 9.0 -715.977 0.0 9.0 0.0 -10.175 -9.0 2.92 -9.0 0.0 -9 -NULL -60 1969-12-31 16:00:12.779 -60.0 NULL 60 -60 0 -60.0 -0.0 60.0 -4773.18 0.0 60.0 0.0 -10.175 -60.0 0.438 -60.0 0.0 -60 -NULL -8 1969-12-31 16:00:00.65 -8.0 NULL 8 -8 0 -8.0 -0.0 8.0 -636.424 0.0 8.0 0.0 -10.175 -8.0 3.285 -8.0 0.0 -8 -true -39 1969-12-31 16:00:03.958 -39.0 821UdmGbkEf4j 39 -39 0 -39.0 -0.0 39.0 -3102.567 0.0 39.0 0.0 -10.175 -39.0 0.6738461538461539 -39.0 0.0 -39 -NULL -5 1969-12-31 16:00:16.107 -5.0 NULL 5 -5 0 -5.0 -0.0 5.0 -397.765 0.0 5.0 0.0 -10.175 -5.0 5.256 -5.0 0.0 -5 -NULL 16 1969-12-31 16:00:16.02 16.0 NULL -16 16 0 16.0 0.0 -16.0 1272.848 0.0 -16.0 0.0 -10.175 16.0 -1.6425 16.0 0.0 16 -NULL 35 1969-12-31 16:00:00.546 35.0 NULL -35 35 0 35.0 0.0 -35.0 2784.355 0.0 -35.0 0.0 -10.175 35.0 -0.7508571428571429 35.0 0.0 35 -NULL 8 1969-12-31 16:00:01.653 8.0 NULL -8 8 0 8.0 0.0 -8.0 636.424 0.0 -8.0 0.0 -10.175 8.0 -3.285 8.0 0.0 8 -true 5 1969-12-31 16:00:08.493 5.0 1cGVWH7n1QU -5 5 0 5.0 0.0 -5.0 397.765 0.0 -5.0 0.0 -10.175 5.0 -5.256 5.0 0.0 5 -NULL -2 1969-12-31 16:00:11.181 -2.0 NULL 2 -2 0 -2.0 -0.0 2.0 -159.106 0.0 2.0 0.0 -10.175 -2.0 13.14 -2.0 0.0 -2 -NULL -20 1969-12-31 16:00:00.51 -20.0 NULL 20 -20 0 -20.0 -0.0 20.0 -1591.06 0.0 20.0 0.0 -10.175 -20.0 1.314 -20.0 0.0 -20 -NULL -22 1969-12-31 16:00:02.221 -22.0 NULL 22 -22 0 -22.0 -0.0 22.0 -1750.166 0.0 22.0 0.0 -10.175 -22.0 1.1945454545454546 -22.0 0.0 -22 -true 2 1969-12-31 16:00:15.071 2.0 821UdmGbkEf4j -2 2 0 2.0 0.0 -2.0 159.106 0.0 -2.0 0.0 -10.175 2.0 -13.14 2.0 0.0 2 -NULL 14 1969-12-31 16:00:10.189 14.0 NULL -14 14 0 14.0 0.0 -14.0 1113.742 0.0 -14.0 0.0 -10.175 14.0 -1.8771428571428572 14.0 0.0 14 -true -12 1969-12-31 16:00:04.049 -12.0 1cGVWH7n1QU 12 -12 0 -12.0 -0.0 12.0 -954.636 0.0 12.0 0.0 -10.175 -12.0 2.19 -12.0 0.0 -12 -true 49 1969-12-31 16:00:07.091 49.0 1cGVWH7n1QU -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 -true 37 1969-12-31 16:00:15.549 37.0 821UdmGbkEf4j -37 37 0 37.0 0.0 -37.0 2943.461 0.0 -37.0 0.0 -10.175 37.0 -0.7102702702702703 37.0 0.0 37 -true -52 1969-12-31 16:00:09.062 -52.0 1cGVWH7n1QU 52 -52 0 -52.0 -0.0 52.0 -4136.755999999999 0.0 52.0 0.0 -10.175 -52.0 0.5053846153846154 -52.0 0.0 -52 -NULL -5 1969-12-31 16:00:16.344 -5.0 NULL 5 -5 0 -5.0 -0.0 5.0 -397.765 0.0 5.0 0.0 -10.175 -5.0 5.256 -5.0 0.0 -5 -true 8 1969-12-31 16:00:05.063 8.0 cvLH6Eat2yFsyy7p -8 8 0 8.0 0.0 -8.0 636.424 0.0 -8.0 0.0 -10.175 8.0 -3.285 8.0 0.0 8 -true -28 1969-12-31 16:00:01.475 -28.0 cvLH6Eat2yFsyy7p 28 -28 0 -28.0 -0.0 28.0 -2227.484 0.0 28.0 0.0 -10.175 -28.0 0.9385714285714286 -28.0 0.0 -28 -NULL 11 1969-12-31 16:00:08.857 11.0 NULL -11 11 0 11.0 0.0 -11.0 875.083 0.0 -11.0 0.0 -10.175 11.0 -2.389090909090909 11.0 0.0 11 -true 44 1969-12-31 16:00:12.47 44.0 821UdmGbkEf4j -44 44 0 44.0 0.0 -44.0 3500.332 0.0 -44.0 0.0 -10.175 44.0 -0.5972727272727273 44.0 0.0 44 -NULL -57 1969-12-31 16:00:15.39 -57.0 NULL 57 -57 0 -57.0 -0.0 57.0 -4534.521 0.0 57.0 0.0 -10.175 -57.0 0.4610526315789474 -57.0 0.0 -57 -NULL -53 1969-12-31 16:00:06.935 -53.0 NULL 53 -53 0 -53.0 -0.0 53.0 -4216.309 0.0 53.0 0.0 -10.175 -53.0 0.4958490566037736 -53.0 0.0 -53 -NULL 7 1969-12-31 16:00:13.807 7.0 NULL -7 7 0 7.0 0.0 -7.0 556.871 0.0 -7.0 0.0 -10.175 7.0 -3.7542857142857144 7.0 0.0 7 -NULL -45 1969-12-31 16:00:09.332 -45.0 NULL 45 -45 0 -45.0 -0.0 45.0 -3579.8849999999998 0.0 45.0 0.0 -10.175 -45.0 0.5840000000000001 -45.0 0.0 -45 -true -52 1969-12-31 16:00:02.242 -52.0 cvLH6Eat2yFsyy7p 52 -52 0 -52.0 -0.0 52.0 -4136.755999999999 0.0 52.0 0.0 -10.175 -52.0 0.5053846153846154 -52.0 0.0 -52 -true 24 1969-12-31 16:00:11.867 24.0 821UdmGbkEf4j -24 24 0 24.0 0.0 -24.0 1909.272 0.0 -24.0 0.0 -10.175 24.0 -1.095 24.0 0.0 24 -NULL 55 1969-12-31 16:00:01.229 55.0 NULL -55 55 0 55.0 0.0 -55.0 4375.415 0.0 -55.0 0.0 -10.175 55.0 -0.47781818181818186 55.0 0.0 55 -true 41 1969-12-31 16:00:05.885 41.0 cvLH6Eat2yFsyy7p -41 41 0 41.0 0.0 -41.0 3261.673 0.0 -41.0 0.0 -10.175 41.0 -0.6409756097560976 41.0 0.0 41 -true -30 1969-12-31 16:00:11.913 -30.0 cvLH6Eat2yFsyy7p 30 -30 0 -30.0 -0.0 30.0 -2386.59 0.0 30.0 0.0 -10.175 -30.0 0.876 -30.0 0.0 -30 -true -2 1969-12-31 16:00:10.495 -2.0 821UdmGbkEf4j 2 -2 0 -2.0 -0.0 2.0 -159.106 0.0 2.0 0.0 -10.175 -2.0 13.14 -2.0 0.0 -2 -true 17 1969-12-31 16:00:00.499 17.0 1cGVWH7n1QU -17 17 0 17.0 0.0 -17.0 1352.4009999999998 0.0 -17.0 0.0 -10.175 17.0 -1.5458823529411765 17.0 0.0 17 -NULL 25 1969-12-31 16:00:12.469 25.0 NULL -25 25 0 25.0 0.0 -25.0 1988.8249999999998 0.0 -25.0 0.0 -10.175 25.0 -1.0512000000000001 25.0 0.0 25 -NULL 37 1969-12-31 16:00:09.934 37.0 NULL -37 37 0 37.0 0.0 -37.0 2943.461 0.0 -37.0 0.0 -10.175 37.0 -0.7102702702702703 37.0 0.0 37 -NULL -9 1969-12-31 16:00:04.372 -9.0 NULL 9 -9 0 -9.0 -0.0 9.0 -715.977 0.0 9.0 0.0 -10.175 -9.0 2.92 -9.0 0.0 -9 -NULL 37 1969-12-31 16:00:16.313 37.0 NULL -37 37 0 37.0 0.0 -37.0 2943.461 0.0 -37.0 0.0 -10.175 37.0 -0.7102702702702703 37.0 0.0 37 -NULL 13 1969-12-31 16:00:12.021 13.0 NULL -13 13 0 13.0 0.0 -13.0 1034.1889999999999 0.0 -13.0 0.0 -10.175 13.0 -2.0215384615384617 13.0 0.0 13 -true -62 1969-12-31 16:00:02.123 -62.0 1cGVWH7n1QU 62 -62 0 -62.0 -0.0 62.0 -4932.286 0.0 62.0 0.0 -10.175 -62.0 0.4238709677419355 -62.0 0.0 -62 -true -4 1969-12-31 16:00:09.922 -4.0 821UdmGbkEf4j 4 -4 0 -4.0 -0.0 4.0 -318.212 0.0 4.0 0.0 -10.175 -4.0 6.57 -4.0 0.0 -4 -true 58 1969-12-31 16:00:02.09 58.0 1cGVWH7n1QU -58 58 0 58.0 0.0 -58.0 4614.074 0.0 -58.0 0.0 -10.175 58.0 -0.4531034482758621 58.0 0.0 58 -true -9 1969-12-31 16:00:11.732 -9.0 1cGVWH7n1QU 9 -9 0 -9.0 -0.0 9.0 -715.977 0.0 9.0 0.0 -10.175 -9.0 2.92 -9.0 0.0 -9 -NULL -23 1969-12-31 16:00:04.124 -23.0 NULL 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -NULL -29 1969-12-31 16:00:04.004 -29.0 NULL 29 -29 0 -29.0 -0.0 29.0 -2307.037 0.0 29.0 0.0 -10.175 -29.0 0.9062068965517242 -29.0 0.0 -29 -NULL 36 1969-12-31 16:00:07.504 36.0 NULL -36 36 0 36.0 0.0 -36.0 2863.908 0.0 -36.0 0.0 -10.175 36.0 -0.73 36.0 0.0 36 -NULL -53 1969-12-31 16:00:09.061 -53.0 NULL 53 -53 0 -53.0 -0.0 53.0 -4216.309 0.0 53.0 0.0 -10.175 -53.0 0.4958490566037736 -53.0 0.0 -53 -true 54 1969-12-31 16:00:05.961 54.0 cvLH6Eat2yFsyy7p -54 54 0 54.0 0.0 -54.0 4295.862 0.0 -54.0 0.0 -10.175 54.0 -0.4866666666666667 54.0 0.0 54 -NULL -53 1969-12-31 16:00:10.631 -53.0 NULL 53 -53 0 -53.0 -0.0 53.0 -4216.309 0.0 53.0 0.0 -10.175 -53.0 0.4958490566037736 -53.0 0.0 -53 -true -52 1969-12-31 16:00:09.929 -52.0 821UdmGbkEf4j 52 -52 0 -52.0 -0.0 52.0 -4136.755999999999 0.0 52.0 0.0 -10.175 -52.0 0.5053846153846154 -52.0 0.0 -52 -NULL 28 1969-12-31 16:00:12.853 28.0 NULL -28 28 0 28.0 0.0 -28.0 2227.484 0.0 -28.0 0.0 -10.175 28.0 -0.9385714285714286 28.0 0.0 28 -true -61 1969-12-31 16:00:12.045 -61.0 1cGVWH7n1QU 61 -61 0 -61.0 -0.0 61.0 -4852.733 0.0 61.0 0.0 -10.175 -61.0 0.4308196721311476 -61.0 0.0 -61 -true -56 1969-12-31 16:00:16.37 -56.0 cvLH6Eat2yFsyy7p 56 -56 0 -56.0 -0.0 56.0 -4454.968 0.0 56.0 0.0 -10.175 -56.0 0.4692857142857143 -56.0 0.0 -56 -true 28 1969-12-31 16:00:16.036 28.0 821UdmGbkEf4j -28 28 0 28.0 0.0 -28.0 2227.484 0.0 -28.0 0.0 -10.175 28.0 -0.9385714285714286 28.0 0.0 28 -true -5 1969-12-31 16:00:04.033 -5.0 cvLH6Eat2yFsyy7p 5 -5 0 -5.0 -0.0 5.0 -397.765 0.0 5.0 0.0 -10.175 -5.0 5.256 -5.0 0.0 -5 -NULL -11 1969-12-31 16:00:07.946 -11.0 NULL 11 -11 0 -11.0 -0.0 11.0 -875.083 0.0 11.0 0.0 -10.175 -11.0 2.389090909090909 -11.0 0.0 -11 -NULL 62 1969-12-31 16:00:12.36 62.0 NULL -62 62 0 62.0 0.0 -62.0 4932.286 0.0 -62.0 0.0 -10.175 62.0 -0.4238709677419355 62.0 0.0 62 -true 35 1969-12-31 16:00:09.647 35.0 1cGVWH7n1QU -35 35 0 35.0 0.0 -35.0 2784.355 0.0 -35.0 0.0 -10.175 35.0 -0.7508571428571429 35.0 0.0 35 -NULL -14 1969-12-31 16:00:04.772 -14.0 NULL 14 -14 0 -14.0 -0.0 14.0 -1113.742 0.0 14.0 0.0 -10.175 -14.0 1.8771428571428572 -14.0 0.0 -14 -true -9 1969-12-31 16:00:03.805 -9.0 cvLH6Eat2yFsyy7p 9 -9 0 -9.0 -0.0 9.0 -715.977 0.0 9.0 0.0 -10.175 -9.0 2.92 -9.0 0.0 -9 -true -34 1969-12-31 16:00:15.726 -34.0 821UdmGbkEf4j 34 -34 0 -34.0 -0.0 34.0 -2704.8019999999997 0.0 34.0 0.0 -10.175 -34.0 0.7729411764705882 -34.0 0.0 -34 -true -1 1969-12-31 16:00:00.839 -1.0 1cGVWH7n1QU 1 -1 0 -1.0 -0.0 1.0 -79.553 0.0 1.0 0.0 -10.175 -1.0 26.28 -1.0 0.0 -1 -NULL -10 1969-12-31 16:00:12.682 -10.0 NULL 10 -10 0 -10.0 -0.0 10.0 -795.53 0.0 10.0 0.0 -10.175 -10.0 2.628 -10.0 0.0 -10 -true 8 1969-12-31 16:00:07.452 8.0 1cGVWH7n1QU -8 8 0 8.0 0.0 -8.0 636.424 0.0 -8.0 0.0 -10.175 8.0 -3.285 8.0 0.0 8 -NULL -57 1969-12-31 16:00:04.659 -57.0 NULL 57 -57 0 -57.0 -0.0 57.0 -4534.521 0.0 57.0 0.0 -10.175 -57.0 0.4610526315789474 -57.0 0.0 -57 -NULL -19 1969-12-31 16:00:03.446 -19.0 NULL 19 -19 0 -19.0 -0.0 19.0 -1511.507 0.0 19.0 0.0 -10.175 -19.0 1.3831578947368421 -19.0 0.0 -19 -true -64 1969-12-31 16:00:04.662 -64.0 821UdmGbkEf4j 64 -64 0 -64.0 -0.0 64.0 -5091.392 0.0 64.0 0.0 -10.175 -64.0 0.410625 -64.0 0.0 -64 -true -9 1969-12-31 16:00:02.666 -9.0 cvLH6Eat2yFsyy7p 9 -9 0 -9.0 -0.0 9.0 -715.977 0.0 9.0 0.0 -10.175 -9.0 2.92 -9.0 0.0 -9 -true -45 1969-12-31 16:00:09.426 -45.0 cvLH6Eat2yFsyy7p 45 -45 0 -45.0 -0.0 45.0 -3579.8849999999998 0.0 45.0 0.0 -10.175 -45.0 0.5840000000000001 -45.0 0.0 -45 -NULL -16 1969-12-31 16:00:05.487 -16.0 NULL 16 -16 0 -16.0 -0.0 16.0 -1272.848 0.0 16.0 0.0 -10.175 -16.0 1.6425 -16.0 0.0 -16 -NULL -38 1969-12-31 16:00:09.402 -38.0 NULL 38 -38 0 -38.0 -0.0 38.0 -3023.014 0.0 38.0 0.0 -10.175 -38.0 0.6915789473684211 -38.0 0.0 -38 -NULL 49 1969-12-31 16:00:15.113 49.0 NULL -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 -NULL -55 1969-12-31 16:00:05.793 -55.0 NULL 55 -55 0 -55.0 -0.0 55.0 -4375.415 0.0 55.0 0.0 -10.175 -55.0 0.47781818181818186 -55.0 0.0 -55 -true 30 1969-12-31 16:00:04.936 30.0 cvLH6Eat2yFsyy7p -30 30 0 30.0 0.0 -30.0 2386.59 0.0 -30.0 0.0 -10.175 30.0 -0.876 30.0 0.0 30 -true 26 1969-12-31 16:00:03.158 26.0 1cGVWH7n1QU -26 26 0 26.0 0.0 -26.0 2068.3779999999997 0.0 -26.0 0.0 -10.175 26.0 -1.0107692307692309 26.0 0.0 26 -NULL 4 1969-12-31 16:00:15.331 4.0 NULL -4 4 0 4.0 0.0 -4.0 318.212 0.0 -4.0 0.0 -10.175 4.0 -6.57 4.0 0.0 4 -true 4 1969-12-31 16:00:15.932 4.0 821UdmGbkEf4j -4 4 0 4.0 0.0 -4.0 318.212 0.0 -4.0 0.0 -10.175 4.0 -6.57 4.0 0.0 4 -NULL -62 1969-12-31 16:00:03.85 -62.0 NULL 62 -62 0 -62.0 -0.0 62.0 -4932.286 0.0 62.0 0.0 -10.175 -62.0 0.4238709677419355 -62.0 0.0 -62 -NULL 22 1969-12-31 16:00:09.697 22.0 NULL -22 22 0 22.0 0.0 -22.0 1750.166 0.0 -22.0 0.0 -10.175 22.0 -1.1945454545454546 22.0 0.0 22 -true -5 1969-12-31 16:00:15.054 -5.0 1cGVWH7n1QU 5 -5 0 -5.0 -0.0 5.0 -397.765 0.0 5.0 0.0 -10.175 -5.0 5.256 -5.0 0.0 -5 -NULL 27 1969-12-31 16:00:00.614 27.0 NULL -27 27 0 27.0 0.0 -27.0 2147.931 0.0 -27.0 0.0 -10.175 27.0 -0.9733333333333334 27.0 0.0 27 -NULL 21 1969-12-31 16:00:01.707 21.0 NULL -21 21 0 21.0 0.0 -21.0 1670.6129999999998 0.0 -21.0 0.0 -10.175 21.0 -1.2514285714285716 21.0 0.0 21 -true 11 1969-12-31 16:00:03.211 11.0 1cGVWH7n1QU -11 11 0 11.0 0.0 -11.0 875.083 0.0 -11.0 0.0 -10.175 11.0 -2.389090909090909 11.0 0.0 11 -NULL 52 1969-12-31 16:00:01.062 52.0 NULL -52 52 0 52.0 0.0 -52.0 4136.755999999999 0.0 -52.0 0.0 -10.175 52.0 -0.5053846153846154 52.0 0.0 52 -true 42 1969-12-31 16:00:00.367 42.0 cvLH6Eat2yFsyy7p -42 42 0 42.0 0.0 -42.0 3341.2259999999997 0.0 -42.0 0.0 -10.175 42.0 -0.6257142857142858 42.0 0.0 42 -NULL 48 1969-12-31 16:00:07.257 48.0 NULL -48 48 0 48.0 0.0 -48.0 3818.544 0.0 -48.0 0.0 -10.175 48.0 -0.5475 48.0 0.0 48 -NULL -59 1969-12-31 16:00:04.16 -59.0 NULL 59 -59 0 -59.0 -0.0 59.0 -4693.6269999999995 0.0 59.0 0.0 -10.175 -59.0 0.44542372881355935 -59.0 0.0 -59 -NULL -22 1969-12-31 16:00:00.229 -22.0 NULL 22 -22 0 -22.0 -0.0 22.0 -1750.166 0.0 22.0 0.0 -10.175 -22.0 1.1945454545454546 -22.0 0.0 -22 -NULL -24 1969-12-31 16:00:07.642 -24.0 NULL 24 -24 0 -24.0 -0.0 24.0 -1909.272 0.0 24.0 0.0 -10.175 -24.0 1.095 -24.0 0.0 -24 -NULL 14 1969-12-31 16:00:06.731 14.0 NULL -14 14 0 14.0 0.0 -14.0 1113.742 0.0 -14.0 0.0 -10.175 14.0 -1.8771428571428572 14.0 0.0 14 -NULL -60 1969-12-31 16:00:10.618 -60.0 NULL 60 -60 0 -60.0 -0.0 60.0 -4773.18 0.0 60.0 0.0 -10.175 -60.0 0.438 -60.0 0.0 -60 -NULL -4 1969-12-31 16:00:07.712 -4.0 NULL 4 -4 0 -4.0 -0.0 4.0 -318.212 0.0 4.0 0.0 -10.175 -4.0 6.57 -4.0 0.0 -4 -true 15 1969-12-31 16:00:16.232 15.0 cvLH6Eat2yFsyy7p -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -true -4 1969-12-31 16:00:03.293 -4.0 1cGVWH7n1QU 4 -4 0 -4.0 -0.0 4.0 -318.212 0.0 4.0 0.0 -10.175 -4.0 6.57 -4.0 0.0 -4 -NULL -8 1969-12-31 16:00:08.252 -8.0 NULL 8 -8 0 -8.0 -0.0 8.0 -636.424 0.0 8.0 0.0 -10.175 -8.0 3.285 -8.0 0.0 -8 -true -46 1969-12-31 16:00:03.399 -46.0 1cGVWH7n1QU 46 -46 0 -46.0 -0.0 46.0 -3659.438 0.0 46.0 0.0 -10.175 -46.0 0.571304347826087 -46.0 0.0 -46 -NULL -13 1969-12-31 16:00:15.003 -13.0 NULL 13 -13 0 -13.0 -0.0 13.0 -1034.1889999999999 0.0 13.0 0.0 -10.175 -13.0 2.0215384615384617 -13.0 0.0 -13 -true -23 1969-12-31 16:00:05.716 -23.0 821UdmGbkEf4j 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -NULL 25 1969-12-31 16:00:01.111 25.0 NULL -25 25 0 25.0 0.0 -25.0 1988.8249999999998 0.0 -25.0 0.0 -10.175 25.0 -1.0512000000000001 25.0 0.0 25 -NULL -20 1969-12-31 16:00:15.793 -20.0 NULL 20 -20 0 -20.0 -0.0 20.0 -1591.06 0.0 20.0 0.0 -10.175 -20.0 1.314 -20.0 0.0 -20 -NULL 53 1969-12-31 16:00:15.38 53.0 NULL -53 53 0 53.0 0.0 -53.0 4216.309 0.0 -53.0 0.0 -10.175 53.0 -0.4958490566037736 53.0 0.0 53 -NULL -1 1969-12-31 16:00:11.049 -1.389 NULL 1 -1 0 -1.3890000581741333 -0.0 1.3890000581741333 -110.49912162792683 0.0 1.3890000581741333 0.0 -10.175 -1.3890000581741333 18.920085600677048 -1.389 0.0 -1 -NULL 25 1969-12-31 16:00:08.234 25.0 NULL -25 25 0 25.0 0.0 -25.0 1988.8249999999998 0.0 -25.0 0.0 -10.175 25.0 -1.0512000000000001 25.0 0.0 25 -true 55 1969-12-31 16:00:08.748 55.0 821UdmGbkEf4j -55 55 0 55.0 0.0 -55.0 4375.415 0.0 -55.0 0.0 -10.175 55.0 -0.47781818181818186 55.0 0.0 55 -NULL -49 1969-12-31 16:00:02.325 -49.0 NULL 49 -49 0 -49.0 -0.0 49.0 -3898.0969999999998 0.0 49.0 0.0 -10.175 -49.0 0.5363265306122449 -49.0 0.0 -49 -NULL 54 1969-12-31 16:00:08.667 54.0 NULL -54 54 0 54.0 0.0 -54.0 4295.862 0.0 -54.0 0.0 -10.175 54.0 -0.4866666666666667 54.0 0.0 54 -NULL 11 1969-12-31 16:00:15.602 11.0 NULL -11 11 0 11.0 0.0 -11.0 875.083 0.0 -11.0 0.0 -10.175 11.0 -2.389090909090909 11.0 0.0 11 -NULL -20 1969-12-31 16:00:09.416 -20.0 NULL 20 -20 0 -20.0 -0.0 20.0 -1591.06 0.0 20.0 0.0 -10.175 -20.0 1.314 -20.0 0.0 -20 -NULL -39 1969-12-31 16:00:00.958 -39.0 NULL 39 -39 0 -39.0 -0.0 39.0 -3102.567 0.0 39.0 0.0 -10.175 -39.0 0.6738461538461539 -39.0 0.0 -39 -NULL 22 1969-12-31 16:00:00.073 22.0 NULL -22 22 0 22.0 0.0 -22.0 1750.166 0.0 -22.0 0.0 -10.175 22.0 -1.1945454545454546 22.0 0.0 22 -NULL -50 1969-12-31 16:00:14.134 -50.0 NULL 50 -50 0 -50.0 -0.0 50.0 -3977.6499999999996 0.0 50.0 0.0 -10.175 -50.0 0.5256000000000001 -50.0 0.0 -50 -true 40 1969-12-31 16:00:02.488 40.0 821UdmGbkEf4j -40 40 0 40.0 0.0 -40.0 3182.12 0.0 -40.0 0.0 -10.175 40.0 -0.657 40.0 0.0 40 -true 3 1969-12-31 16:00:03.941 3.0 1cGVWH7n1QU -3 3 0 3.0 0.0 -3.0 238.659 0.0 -3.0 0.0 -10.175 3.0 -8.76 3.0 0.0 3 -NULL -17 1969-12-31 16:00:14.733 -17.0 NULL 17 -17 0 -17.0 -0.0 17.0 -1352.4009999999998 0.0 17.0 0.0 -10.175 -17.0 1.5458823529411765 -17.0 0.0 -17 -NULL 2 1969-12-31 16:00:07.654 2.0 NULL -2 2 0 2.0 0.0 -2.0 159.106 0.0 -2.0 0.0 -10.175 2.0 -13.14 2.0 0.0 2 -NULL 62 1969-12-31 16:00:12.762 62.0 NULL -62 62 0 62.0 0.0 -62.0 4932.286 0.0 -62.0 0.0 -10.175 62.0 -0.4238709677419355 62.0 0.0 62 -true 39 1969-12-31 16:00:00.271 39.0 821UdmGbkEf4j -39 39 0 39.0 0.0 -39.0 3102.567 0.0 -39.0 0.0 -10.175 39.0 -0.6738461538461539 39.0 0.0 39 -true -54 1969-12-31 16:00:03.601 -54.0 821UdmGbkEf4j 54 -54 0 -54.0 -0.0 54.0 -4295.862 0.0 54.0 0.0 -10.175 -54.0 0.4866666666666667 -54.0 0.0 -54 -NULL 46 1969-12-31 16:00:04.102 46.0 NULL -46 46 0 46.0 0.0 -46.0 3659.438 0.0 -46.0 0.0 -10.175 46.0 -0.571304347826087 46.0 0.0 46 -true 14 1969-12-31 16:00:07.812 14.0 cvLH6Eat2yFsyy7p -14 14 0 14.0 0.0 -14.0 1113.742 0.0 -14.0 0.0 -10.175 14.0 -1.8771428571428572 14.0 0.0 14 -NULL 6 1969-12-31 16:00:06.061 6.0 NULL -6 6 0 6.0 0.0 -6.0 477.318 0.0 -6.0 0.0 -10.175 6.0 -4.38 6.0 0.0 6 -NULL -6 1969-12-31 16:00:10.435 -6.0 NULL 6 -6 0 -6.0 -0.0 6.0 -477.318 0.0 6.0 0.0 -10.175 -6.0 4.38 -6.0 0.0 -6 -true -55 1969-12-31 16:00:02.71 -55.0 821UdmGbkEf4j 55 -55 0 -55.0 -0.0 55.0 -4375.415 0.0 55.0 0.0 -10.175 -55.0 0.47781818181818186 -55.0 0.0 -55 -true 39 1969-12-31 16:00:00.078 39.0 cvLH6Eat2yFsyy7p -39 39 0 39.0 0.0 -39.0 3102.567 0.0 -39.0 0.0 -10.175 39.0 -0.6738461538461539 39.0 0.0 39 -true 30 1969-12-31 16:00:07.902 30.0 821UdmGbkEf4j -30 30 0 30.0 0.0 -30.0 2386.59 0.0 -30.0 0.0 -10.175 30.0 -0.876 30.0 0.0 30 -NULL 44 1969-12-31 16:00:01.54 44.0 NULL -44 44 0 44.0 0.0 -44.0 3500.332 0.0 -44.0 0.0 -10.175 44.0 -0.5972727272727273 44.0 0.0 44 -true -8 1969-12-31 16:00:00.532 -8.0 cvLH6Eat2yFsyy7p 8 -8 0 -8.0 -0.0 8.0 -636.424 0.0 8.0 0.0 -10.175 -8.0 3.285 -8.0 0.0 -8 -true -28 1969-12-31 16:00:10.554 -28.0 821UdmGbkEf4j 28 -28 0 -28.0 -0.0 28.0 -2227.484 0.0 28.0 0.0 -10.175 -28.0 0.9385714285714286 -28.0 0.0 -28 -true 38 1969-12-31 16:00:03.145 38.0 821UdmGbkEf4j -38 38 0 38.0 0.0 -38.0 3023.014 0.0 -38.0 0.0 -10.175 38.0 -0.6915789473684211 38.0 0.0 38 -NULL -31 1969-12-31 16:00:13.087 -31.0 NULL 31 -31 0 -31.0 -0.0 31.0 -2466.143 0.0 31.0 0.0 -10.175 -31.0 0.847741935483871 -31.0 0.0 -31 -NULL -15 1969-12-31 16:00:15.186 -15.0 NULL 15 -15 0 -15.0 -0.0 15.0 -1193.295 0.0 15.0 0.0 -10.175 -15.0 1.752 -15.0 0.0 -15 -true -41 1969-12-31 16:00:05.707 -41.0 cvLH6Eat2yFsyy7p 41 -41 0 -41.0 -0.0 41.0 -3261.673 0.0 41.0 0.0 -10.175 -41.0 0.6409756097560976 -41.0 0.0 -41 -NULL 18 1969-12-31 16:00:05.804 18.0 NULL -18 18 0 18.0 0.0 -18.0 1431.954 0.0 -18.0 0.0 -10.175 18.0 -1.46 18.0 0.0 18 -NULL 44 1969-12-31 16:00:03.211 44.0 NULL -44 44 0 44.0 0.0 -44.0 3500.332 0.0 -44.0 0.0 -10.175 44.0 -0.5972727272727273 44.0 0.0 44 -true -13 1969-12-31 16:00:02.166 -13.0 821UdmGbkEf4j 13 -13 0 -13.0 -0.0 13.0 -1034.1889999999999 0.0 13.0 0.0 -10.175 -13.0 2.0215384615384617 -13.0 0.0 -13 -true -21 1969-12-31 16:00:10.129 -21.0 821UdmGbkEf4j 21 -21 0 -21.0 -0.0 21.0 -1670.6129999999998 0.0 21.0 0.0 -10.175 -21.0 1.2514285714285716 -21.0 0.0 -21 -NULL -26 1969-12-31 16:00:02.445 -26.0 NULL 26 -26 0 -26.0 -0.0 26.0 -2068.3779999999997 0.0 26.0 0.0 -10.175 -26.0 1.0107692307692309 -26.0 0.0 -26 -true 53 1969-12-31 16:00:03.287 53.0 1cGVWH7n1QU -53 53 0 53.0 0.0 -53.0 4216.309 0.0 -53.0 0.0 -10.175 53.0 -0.4958490566037736 53.0 0.0 53 -NULL -30 1969-12-31 16:00:00.137 -30.0 NULL 30 -30 0 -30.0 -0.0 30.0 -2386.59 0.0 30.0 0.0 -10.175 -30.0 0.876 -30.0 0.0 -30 -NULL 50 1969-12-31 16:00:06.371 50.0 NULL -50 50 0 50.0 0.0 -50.0 3977.6499999999996 0.0 -50.0 0.0 -10.175 50.0 -0.5256000000000001 50.0 0.0 50 -true 7 1969-12-31 16:00:03.526 7.0 1cGVWH7n1QU -7 7 0 7.0 0.0 -7.0 556.871 0.0 -7.0 0.0 -10.175 7.0 -3.7542857142857144 7.0 0.0 7 -NULL -31 1969-12-31 16:00:15.169 -31.0 NULL 31 -31 0 -31.0 -0.0 31.0 -2466.143 0.0 31.0 0.0 -10.175 -31.0 0.847741935483871 -31.0 0.0 -31 -true 9 1969-12-31 16:00:07.037 9.0 1cGVWH7n1QU -9 9 0 9.0 0.0 -9.0 715.977 0.0 -9.0 0.0 -10.175 9.0 -2.92 9.0 0.0 9 -true -31 1969-12-31 16:00:00.572 -31.0 1cGVWH7n1QU 31 -31 0 -31.0 -0.0 31.0 -2466.143 0.0 31.0 0.0 -10.175 -31.0 0.847741935483871 -31.0 0.0 -31 -NULL 17 1969-12-31 16:00:13.008 17.0 NULL -17 17 0 17.0 0.0 -17.0 1352.4009999999998 0.0 -17.0 0.0 -10.175 17.0 -1.5458823529411765 17.0 0.0 17 -NULL -48 1969-12-31 16:00:05.978 -48.0 NULL 48 -48 0 -48.0 -0.0 48.0 -3818.544 0.0 48.0 0.0 -10.175 -48.0 0.5475 -48.0 0.0 -48 -NULL -37 1969-12-31 16:00:15.979 -37.0 NULL 37 -37 0 -37.0 -0.0 37.0 -2943.461 0.0 37.0 0.0 -10.175 -37.0 0.7102702702702703 -37.0 0.0 -37 -NULL 41 1969-12-31 16:00:00.469 41.0 NULL -41 41 0 41.0 0.0 -41.0 3261.673 0.0 -41.0 0.0 -10.175 41.0 -0.6409756097560976 41.0 0.0 41 -true 3 1969-12-31 16:00:05.988 3.0 821UdmGbkEf4j -3 3 0 3.0 0.0 -3.0 238.659 0.0 -3.0 0.0 -10.175 3.0 -8.76 3.0 0.0 3 -NULL -39 1969-12-31 16:00:09.101 -39.0 NULL 39 -39 0 -39.0 -0.0 39.0 -3102.567 0.0 39.0 0.0 -10.175 -39.0 0.6738461538461539 -39.0 0.0 -39 -true -62 1969-12-31 16:00:04.978 -62.0 cvLH6Eat2yFsyy7p 62 -62 0 -62.0 -0.0 62.0 -4932.286 0.0 62.0 0.0 -10.175 -62.0 0.4238709677419355 -62.0 0.0 -62 -NULL -44 1969-12-31 16:00:14.848 -44.0 NULL 44 -44 0 -44.0 -0.0 44.0 -3500.332 0.0 44.0 0.0 -10.175 -44.0 0.5972727272727273 -44.0 0.0 -44 -NULL 24 1969-12-31 16:00:13.787 24.0 NULL -24 24 0 24.0 0.0 -24.0 1909.272 0.0 -24.0 0.0 -10.175 24.0 -1.095 24.0 0.0 24 -NULL 38 1969-12-31 16:00:14.648 38.0 NULL -38 38 0 38.0 0.0 -38.0 3023.014 0.0 -38.0 0.0 -10.175 38.0 -0.6915789473684211 38.0 0.0 38 -NULL -37 1969-12-31 16:00:14.662 -37.0 NULL 37 -37 0 -37.0 -0.0 37.0 -2943.461 0.0 37.0 0.0 -10.175 -37.0 0.7102702702702703 -37.0 0.0 -37 -true -14 1969-12-31 16:00:14.341 -14.0 821UdmGbkEf4j 14 -14 0 -14.0 -0.0 14.0 -1113.742 0.0 14.0 0.0 -10.175 -14.0 1.8771428571428572 -14.0 0.0 -14 -NULL 54 1969-12-31 16:00:00.546 54.0 NULL -54 54 0 54.0 0.0 -54.0 4295.862 0.0 -54.0 0.0 -10.175 54.0 -0.4866666666666667 54.0 0.0 54 -NULL -10 1969-12-31 16:00:03.043 -10.0 NULL 10 -10 0 -10.0 -0.0 10.0 -795.53 0.0 10.0 0.0 -10.175 -10.0 2.628 -10.0 0.0 -10 -NULL 30 1969-12-31 16:00:08.554 30.0 NULL -30 30 0 30.0 0.0 -30.0 2386.59 0.0 -30.0 0.0 -10.175 30.0 -0.876 30.0 0.0 30 -true -4 1969-12-31 16:00:14.16 -4.0 1cGVWH7n1QU 4 -4 0 -4.0 -0.0 4.0 -318.212 0.0 4.0 0.0 -10.175 -4.0 6.57 -4.0 0.0 -4 -NULL -49 1969-12-31 16:00:05.814 -49.0 NULL 49 -49 0 -49.0 -0.0 49.0 -3898.0969999999998 0.0 49.0 0.0 -10.175 -49.0 0.5363265306122449 -49.0 0.0 -49 -true 49 1969-12-31 16:00:00.662 49.0 cvLH6Eat2yFsyy7p -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 -true -10 1969-12-31 16:00:00.082 -10.0 1cGVWH7n1QU 10 -10 0 -10.0 -0.0 10.0 -795.53 0.0 10.0 0.0 -10.175 -10.0 2.628 -10.0 0.0 -10 -true 1 1969-12-31 16:00:02.594 1.0 cvLH6Eat2yFsyy7p -1 1 0 1.0 0.0 -1.0 79.553 0.0 -1.0 0.0 -10.175 1.0 -26.28 1.0 0.0 1 -true -50 1969-12-31 16:00:08.153 -50.0 1cGVWH7n1QU 50 -50 0 -50.0 -0.0 50.0 -3977.6499999999996 0.0 50.0 0.0 -10.175 -50.0 0.5256000000000001 -50.0 0.0 -50 -true -32 1969-12-31 16:00:14.143 -32.0 cvLH6Eat2yFsyy7p 32 -32 0 -32.0 -0.0 32.0 -2545.696 0.0 32.0 0.0 -10.175 -32.0 0.82125 -32.0 0.0 -32 -true -56 1969-12-31 16:00:07.559 -56.0 cvLH6Eat2yFsyy7p 56 -56 0 -56.0 -0.0 56.0 -4454.968 0.0 56.0 0.0 -10.175 -56.0 0.4692857142857143 -56.0 0.0 -56 -true 17 1969-12-31 16:00:00.597 17.0 1cGVWH7n1QU -17 17 0 17.0 0.0 -17.0 1352.4009999999998 0.0 -17.0 0.0 -10.175 17.0 -1.5458823529411765 17.0 0.0 17 -NULL -26 1969-12-31 16:00:06.457 -26.0 NULL 26 -26 0 -26.0 -0.0 26.0 -2068.3779999999997 0.0 26.0 0.0 -10.175 -26.0 1.0107692307692309 -26.0 0.0 -26 -NULL -39 1969-12-31 16:00:10.104 -39.0 NULL 39 -39 0 -39.0 -0.0 39.0 -3102.567 0.0 39.0 0.0 -10.175 -39.0 0.6738461538461539 -39.0 0.0 -39 -true -22 1969-12-31 16:00:08.505 -22.0 821UdmGbkEf4j 22 -22 0 -22.0 -0.0 22.0 -1750.166 0.0 22.0 0.0 -10.175 -22.0 1.1945454545454546 -22.0 0.0 -22 -NULL 14 1969-12-31 16:00:04.516 14.0 NULL -14 14 0 14.0 0.0 -14.0 1113.742 0.0 -14.0 0.0 -10.175 14.0 -1.8771428571428572 14.0 0.0 14 -true -61 1969-12-31 16:00:07.538 -61.0 821UdmGbkEf4j 61 -61 0 -61.0 -0.0 61.0 -4852.733 0.0 61.0 0.0 -10.175 -61.0 0.4308196721311476 -61.0 0.0 -61 -NULL 16 1969-12-31 16:00:08.432 16.0 NULL -16 16 0 16.0 0.0 -16.0 1272.848 0.0 -16.0 0.0 -10.175 16.0 -1.6425 16.0 0.0 16 -true 46 1969-12-31 16:00:00.863 46.0 1cGVWH7n1QU -46 46 0 46.0 0.0 -46.0 3659.438 0.0 -46.0 0.0 -10.175 46.0 -0.571304347826087 46.0 0.0 46 -true -45 1969-12-31 16:00:09.949 -45.0 1cGVWH7n1QU 45 -45 0 -45.0 -0.0 45.0 -3579.8849999999998 0.0 45.0 0.0 -10.175 -45.0 0.5840000000000001 -45.0 0.0 -45 -NULL -47 1969-12-31 16:00:00.436 -47.0 NULL 47 -47 0 -47.0 -0.0 47.0 -3738.991 0.0 47.0 0.0 -10.175 -47.0 0.5591489361702128 -47.0 0.0 -47 -true 5 1969-12-31 16:00:00.467 5.0 cvLH6Eat2yFsyy7p -5 5 0 5.0 0.0 -5.0 397.765 0.0 -5.0 0.0 -10.175 5.0 -5.256 5.0 0.0 5 -true 15 1969-12-31 16:00:09.577 15.0 1cGVWH7n1QU -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -true 28 1969-12-31 16:00:02.496 28.0 cvLH6Eat2yFsyy7p -28 28 0 28.0 0.0 -28.0 2227.484 0.0 -28.0 0.0 -10.175 28.0 -0.9385714285714286 28.0 0.0 28 -NULL 11 1969-12-31 16:00:14.569 11.0 NULL -11 11 0 11.0 0.0 -11.0 875.083 0.0 -11.0 0.0 -10.175 11.0 -2.389090909090909 11.0 0.0 11 -NULL -6 1969-12-31 16:00:12.469 -6.0 NULL 6 -6 0 -6.0 -0.0 6.0 -477.318 0.0 6.0 0.0 -10.175 -6.0 4.38 -6.0 0.0 -6 -true 40 1969-12-31 16:00:01.206 40.0 cvLH6Eat2yFsyy7p -40 40 0 40.0 0.0 -40.0 3182.12 0.0 -40.0 0.0 -10.175 40.0 -0.657 40.0 0.0 40 -true 49 1969-12-31 16:00:12.273 49.0 821UdmGbkEf4j -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 -true -33 1969-12-31 16:00:08.231 -33.0 821UdmGbkEf4j 33 -33 0 -33.0 -0.0 33.0 -2625.249 0.0 33.0 0.0 -10.175 -33.0 0.7963636363636364 -33.0 0.0 -33 -NULL -49 1969-12-31 16:00:11.627 -49.0 NULL 49 -49 0 -49.0 -0.0 49.0 -3898.0969999999998 0.0 49.0 0.0 -10.175 -49.0 0.5363265306122449 -49.0 0.0 -49 -NULL 13 1969-12-31 16:00:14.548 13.0 NULL -13 13 0 13.0 0.0 -13.0 1034.1889999999999 0.0 -13.0 0.0 -10.175 13.0 -2.0215384615384617 13.0 0.0 13 -NULL -35 1969-12-31 16:00:00.8 -35.0 NULL 35 -35 0 -35.0 -0.0 35.0 -2784.355 0.0 35.0 0.0 -10.175 -35.0 0.7508571428571429 -35.0 0.0 -35 -NULL -2 1969-12-31 16:00:14.391 -2.0 NULL 2 -2 0 -2.0 -0.0 2.0 -159.106 0.0 2.0 0.0 -10.175 -2.0 13.14 -2.0 0.0 -2 -true 13 1969-12-31 16:00:07.709 13.0 cvLH6Eat2yFsyy7p -13 13 0 13.0 0.0 -13.0 1034.1889999999999 0.0 -13.0 0.0 -10.175 13.0 -2.0215384615384617 13.0 0.0 13 -true -56 1969-12-31 16:00:00.298 -56.0 1cGVWH7n1QU 56 -56 0 -56.0 -0.0 56.0 -4454.968 0.0 56.0 0.0 -10.175 -56.0 0.4692857142857143 -56.0 0.0 -56 -true 25 1969-12-31 16:00:05.327 25.0 cvLH6Eat2yFsyy7p -25 25 0 25.0 0.0 -25.0 1988.8249999999998 0.0 -25.0 0.0 -10.175 25.0 -1.0512000000000001 25.0 0.0 25 -NULL 62 1969-12-31 16:00:14.982 62.0 NULL -62 62 0 62.0 0.0 -62.0 4932.286 0.0 -62.0 0.0 -10.175 62.0 -0.4238709677419355 62.0 0.0 62 -true 56 1969-12-31 16:00:01.556 56.0 cvLH6Eat2yFsyy7p -56 56 0 56.0 0.0 -56.0 4454.968 0.0 -56.0 0.0 -10.175 56.0 -0.4692857142857143 56.0 0.0 56 -true -24 1969-12-31 16:00:03.519 -24.0 cvLH6Eat2yFsyy7p 24 -24 0 -24.0 -0.0 24.0 -1909.272 0.0 24.0 0.0 -10.175 -24.0 1.095 -24.0 0.0 -24 -NULL 44 1969-12-31 16:00:05.434 44.0 NULL -44 44 0 44.0 0.0 -44.0 3500.332 0.0 -44.0 0.0 -10.175 44.0 -0.5972727272727273 44.0 0.0 44 -NULL -36 1969-12-31 16:00:01.626 -36.0 NULL 36 -36 0 -36.0 -0.0 36.0 -2863.908 0.0 36.0 0.0 -10.175 -36.0 0.73 -36.0 0.0 -36 -NULL -60 1969-12-31 16:00:00.043 -60.0 NULL 60 -60 0 -60.0 -0.0 60.0 -4773.18 0.0 60.0 0.0 -10.175 -60.0 0.438 -60.0 0.0 -60 -NULL 22 1969-12-31 16:00:00.306 22.0 NULL -22 22 0 22.0 0.0 -22.0 1750.166 0.0 -22.0 0.0 -10.175 22.0 -1.1945454545454546 22.0 0.0 22 -true 25 1969-12-31 16:00:13.338 25.0 cvLH6Eat2yFsyy7p -25 25 0 25.0 0.0 -25.0 1988.8249999999998 0.0 -25.0 0.0 -10.175 25.0 -1.0512000000000001 25.0 0.0 25 -true -1 1969-12-31 16:00:16.189 -1.0 cvLH6Eat2yFsyy7p 1 -1 0 -1.0 -0.0 1.0 -79.553 0.0 1.0 0.0 -10.175 -1.0 26.28 -1.0 0.0 -1 -true 13 1969-12-31 16:00:08.549 13.0 821UdmGbkEf4j -13 13 0 13.0 0.0 -13.0 1034.1889999999999 0.0 -13.0 0.0 -10.175 13.0 -2.0215384615384617 13.0 0.0 13 -true 6 1969-12-31 16:00:11.357 6.0 cvLH6Eat2yFsyy7p -6 6 0 6.0 0.0 -6.0 477.318 0.0 -6.0 0.0 -10.175 6.0 -4.38 6.0 0.0 6 -NULL 30 1969-12-31 16:00:11.876 30.0 NULL -30 30 0 30.0 0.0 -30.0 2386.59 0.0 -30.0 0.0 -10.175 30.0 -0.876 30.0 0.0 30 -NULL -23 1969-12-31 16:00:03.888 -23.0 NULL 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -NULL -43 1969-12-31 16:00:06.507 -43.0 NULL 43 -43 0 -43.0 -0.0 43.0 -3420.779 0.0 43.0 0.0 -10.175 -43.0 0.6111627906976744 -43.0 0.0 -43 -NULL 2 1969-12-31 16:00:10.198 2.0 NULL -2 2 0 2.0 0.0 -2.0 159.106 0.0 -2.0 0.0 -10.175 2.0 -13.14 2.0 0.0 2 -NULL 15 1969-12-31 16:00:04.971 15.0 NULL -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -true -56 1969-12-31 16:00:07.316 -56.0 cvLH6Eat2yFsyy7p 56 -56 0 -56.0 -0.0 56.0 -4454.968 0.0 56.0 0.0 -10.175 -56.0 0.4692857142857143 -56.0 0.0 -56 -NULL 33 1969-12-31 16:00:04.063 79.553 NULL -33 33 0 79.5530014038086 0.0 -79.5530014038086 6328.679920677185 0.0 -79.5530014038086 0.0 -10.175 79.5530014038086 -0.33034580136836733 79.553 0.0 33 -true 57 1969-12-31 16:00:13.71 57.0 cvLH6Eat2yFsyy7p -57 57 0 57.0 0.0 -57.0 4534.521 0.0 -57.0 0.0 -10.175 57.0 -0.4610526315789474 57.0 0.0 57 -NULL 27 1969-12-31 16:00:09.971 27.0 NULL -27 27 0 27.0 0.0 -27.0 2147.931 0.0 -27.0 0.0 -10.175 27.0 -0.9733333333333334 27.0 0.0 27 -true 58 1969-12-31 16:00:11.54 58.0 1cGVWH7n1QU -58 58 0 58.0 0.0 -58.0 4614.074 0.0 -58.0 0.0 -10.175 58.0 -0.4531034482758621 58.0 0.0 58 -true -22 1969-12-31 16:00:14.821 -22.0 821UdmGbkEf4j 22 -22 0 -22.0 -0.0 22.0 -1750.166 0.0 22.0 0.0 -10.175 -22.0 1.1945454545454546 -22.0 0.0 -22 -true -40 1969-12-31 16:00:01.965 -40.0 821UdmGbkEf4j 40 -40 0 -40.0 -0.0 40.0 -3182.12 0.0 40.0 0.0 -10.175 -40.0 0.657 -40.0 0.0 -40 -true 33 1969-12-31 16:00:15.655 33.0 cvLH6Eat2yFsyy7p -33 33 0 33.0 0.0 -33.0 2625.249 0.0 -33.0 0.0 -10.175 33.0 -0.7963636363636364 33.0 0.0 33 -true -9 1969-12-31 16:00:10.52 -9.0 821UdmGbkEf4j 9 -9 0 -9.0 -0.0 9.0 -715.977 0.0 9.0 0.0 -10.175 -9.0 2.92 -9.0 0.0 -9 -true 13 1969-12-31 16:00:05.942 13.0 1cGVWH7n1QU -13 13 0 13.0 0.0 -13.0 1034.1889999999999 0.0 -13.0 0.0 -10.175 13.0 -2.0215384615384617 13.0 0.0 13 -true -52 1969-12-31 16:00:14.489 -52.0 1cGVWH7n1QU 52 -52 0 -52.0 -0.0 52.0 -4136.755999999999 0.0 52.0 0.0 -10.175 -52.0 0.5053846153846154 -52.0 0.0 -52 -true 29 1969-12-31 16:00:00.66 29.0 821UdmGbkEf4j -29 29 0 29.0 0.0 -29.0 2307.037 0.0 -29.0 0.0 -10.175 29.0 -0.9062068965517242 29.0 0.0 29 -true -17 1969-12-31 16:00:12.685 -17.0 1cGVWH7n1QU 17 -17 0 -17.0 -0.0 17.0 -1352.4009999999998 0.0 17.0 0.0 -10.175 -17.0 1.5458823529411765 -17.0 0.0 -17 -NULL -19 1969-12-31 16:00:07.935 -19.0 NULL 19 -19 0 -19.0 -0.0 19.0 -1511.507 0.0 19.0 0.0 -10.175 -19.0 1.3831578947368421 -19.0 0.0 -19 -NULL -29 1969-12-31 16:00:10.721 -29.0 NULL 29 -29 0 -29.0 -0.0 29.0 -2307.037 0.0 29.0 0.0 -10.175 -29.0 0.9062068965517242 -29.0 0.0 -29 -true -4 1969-12-31 16:00:03.366 -4.0 821UdmGbkEf4j 4 -4 0 -4.0 -0.0 4.0 -318.212 0.0 4.0 0.0 -10.175 -4.0 6.57 -4.0 0.0 -4 -true 7 1969-12-31 16:00:09.168 7.0 821UdmGbkEf4j -7 7 0 7.0 0.0 -7.0 556.871 0.0 -7.0 0.0 -10.175 7.0 -3.7542857142857144 7.0 0.0 7 -true -6 1969-12-31 16:00:09.052 -6.0 821UdmGbkEf4j 6 -6 0 -6.0 -0.0 6.0 -477.318 0.0 6.0 0.0 -10.175 -6.0 4.38 -6.0 0.0 -6 -NULL -47 1969-12-31 16:00:14.216 -47.0 NULL 47 -47 0 -47.0 -0.0 47.0 -3738.991 0.0 47.0 0.0 -10.175 -47.0 0.5591489361702128 -47.0 0.0 -47 -true 40 1969-12-31 16:00:16.188 40.0 1cGVWH7n1QU -40 40 0 40.0 0.0 -40.0 3182.12 0.0 -40.0 0.0 -10.175 40.0 -0.657 40.0 0.0 40 -true -48 1969-12-31 16:00:04.812 -48.0 821UdmGbkEf4j 48 -48 0 -48.0 -0.0 48.0 -3818.544 0.0 48.0 0.0 -10.175 -48.0 0.5475 -48.0 0.0 -48 -true 54 1969-12-31 16:00:10.663 54.0 821UdmGbkEf4j -54 54 0 54.0 0.0 -54.0 4295.862 0.0 -54.0 0.0 -10.175 54.0 -0.4866666666666667 54.0 0.0 54 -true 25 1969-12-31 16:00:00.803 25.0 cvLH6Eat2yFsyy7p -25 25 0 25.0 0.0 -25.0 1988.8249999999998 0.0 -25.0 0.0 -10.175 25.0 -1.0512000000000001 25.0 0.0 25 -true -64 1969-12-31 16:00:00.631 -64.0 1cGVWH7n1QU 64 -64 0 -64.0 -0.0 64.0 -5091.392 0.0 64.0 0.0 -10.175 -64.0 0.410625 -64.0 0.0 -64 -NULL 54 1969-12-31 16:00:01.733 54.0 NULL -54 54 0 54.0 0.0 -54.0 4295.862 0.0 -54.0 0.0 -10.175 54.0 -0.4866666666666667 54.0 0.0 54 -NULL 21 1969-12-31 16:00:15.732 21.0 NULL -21 21 0 21.0 0.0 -21.0 1670.6129999999998 0.0 -21.0 0.0 -10.175 21.0 -1.2514285714285716 21.0 0.0 21 -true 13 1969-12-31 16:00:09.802 13.0 821UdmGbkEf4j -13 13 0 13.0 0.0 -13.0 1034.1889999999999 0.0 -13.0 0.0 -10.175 13.0 -2.0215384615384617 13.0 0.0 13 -true -20 1969-12-31 16:00:02.242 -20.0 1cGVWH7n1QU 20 -20 0 -20.0 -0.0 20.0 -1591.06 0.0 20.0 0.0 -10.175 -20.0 1.314 -20.0 0.0 -20 -true 18 1969-12-31 16:00:12.339 18.0 cvLH6Eat2yFsyy7p -18 18 0 18.0 0.0 -18.0 1431.954 0.0 -18.0 0.0 -10.175 18.0 -1.46 18.0 0.0 18 -NULL 51 1969-12-31 16:00:06.516 51.0 NULL -51 51 0 51.0 0.0 -51.0 4057.203 0.0 -51.0 0.0 -10.175 51.0 -0.5152941176470588 51.0 0.0 51 -NULL 36 1969-12-31 16:00:13.717 36.0 NULL -36 36 0 36.0 0.0 -36.0 2863.908 0.0 -36.0 0.0 -10.175 36.0 -0.73 36.0 0.0 36 -NULL -61 1969-12-31 16:00:07.405 -61.0 NULL 61 -61 0 -61.0 -0.0 61.0 -4852.733 0.0 61.0 0.0 -10.175 -61.0 0.4308196721311476 -61.0 0.0 -61 -NULL 37 1969-12-31 16:00:00.024 37.0 NULL -37 37 0 37.0 0.0 -37.0 2943.461 0.0 -37.0 0.0 -10.175 37.0 -0.7102702702702703 37.0 0.0 37 -true -57 1969-12-31 16:00:08.471 -57.0 cvLH6Eat2yFsyy7p 57 -57 0 -57.0 -0.0 57.0 -4534.521 0.0 57.0 0.0 -10.175 -57.0 0.4610526315789474 -57.0 0.0 -57 -NULL 14 1969-12-31 16:00:01.873 14.0 NULL -14 14 0 14.0 0.0 -14.0 1113.742 0.0 -14.0 0.0 -10.175 14.0 -1.8771428571428572 14.0 0.0 14 -true -37 1969-12-31 16:00:13.3 -37.0 cvLH6Eat2yFsyy7p 37 -37 0 -37.0 -0.0 37.0 -2943.461 0.0 37.0 0.0 -10.175 -37.0 0.7102702702702703 -37.0 0.0 -37 -NULL 21 1969-12-31 16:00:12.52 21.0 NULL -21 21 0 21.0 0.0 -21.0 1670.6129999999998 0.0 -21.0 0.0 -10.175 21.0 -1.2514285714285716 21.0 0.0 21 -NULL -17 1969-12-31 16:00:06.738 -17.0 NULL 17 -17 0 -17.0 -0.0 17.0 -1352.4009999999998 0.0 17.0 0.0 -10.175 -17.0 1.5458823529411765 -17.0 0.0 -17 -NULL 51 1969-12-31 16:00:00.147 51.0 NULL -51 51 0 51.0 0.0 -51.0 4057.203 0.0 -51.0 0.0 -10.175 51.0 -0.5152941176470588 51.0 0.0 51 -true -29 1969-12-31 16:00:00.02 -29.0 821UdmGbkEf4j 29 -29 0 -29.0 -0.0 29.0 -2307.037 0.0 29.0 0.0 -10.175 -29.0 0.9062068965517242 -29.0 0.0 -29 -NULL -22 1969-12-31 16:00:04.92 -22.0 NULL 22 -22 0 -22.0 -0.0 22.0 -1750.166 0.0 22.0 0.0 -10.175 -22.0 1.1945454545454546 -22.0 0.0 -22 -NULL -42 1969-12-31 16:00:02.88 -42.0 NULL 42 -42 0 -42.0 -0.0 42.0 -3341.2259999999997 0.0 42.0 0.0 -10.175 -42.0 0.6257142857142858 -42.0 0.0 -42 -NULL -63 1969-12-31 16:00:12.188 -63.0 NULL 63 -63 0 -63.0 -0.0 63.0 -5011.839 0.0 63.0 0.0 -10.175 -63.0 0.41714285714285715 -63.0 0.0 -63 -true -60 1969-12-31 16:00:15.188 -60.0 cvLH6Eat2yFsyy7p 60 -60 0 -60.0 -0.0 60.0 -4773.18 0.0 60.0 0.0 -10.175 -60.0 0.438 -60.0 0.0 -60 -true 59 1969-12-31 16:00:04.432 59.0 1cGVWH7n1QU -59 59 0 59.0 0.0 -59.0 4693.6269999999995 0.0 -59.0 0.0 -10.175 59.0 -0.44542372881355935 59.0 0.0 59 -true -48 1969-12-31 16:00:09.393 -48.0 821UdmGbkEf4j 48 -48 0 -48.0 -0.0 48.0 -3818.544 0.0 48.0 0.0 -10.175 -48.0 0.5475 -48.0 0.0 -48 -true 3 1969-12-31 16:00:03.437 3.0 821UdmGbkEf4j -3 3 0 3.0 0.0 -3.0 238.659 0.0 -3.0 0.0 -10.175 3.0 -8.76 3.0 0.0 3 -NULL -32 1969-12-31 16:00:01.093 -32.0 NULL 32 -32 0 -32.0 -0.0 32.0 -2545.696 0.0 32.0 0.0 -10.175 -32.0 0.82125 -32.0 0.0 -32 -NULL 26 1969-12-31 16:00:08.286 26.0 NULL -26 26 0 26.0 0.0 -26.0 2068.3779999999997 0.0 -26.0 0.0 -10.175 26.0 -1.0107692307692309 26.0 0.0 26 -true -27 1969-12-31 16:00:05.617 -27.0 cvLH6Eat2yFsyy7p 27 -27 0 -27.0 -0.0 27.0 -2147.931 0.0 27.0 0.0 -10.175 -27.0 0.9733333333333334 -27.0 0.0 -27 -NULL 1 1969-12-31 16:00:11.949 1.0 NULL -1 1 0 1.0 0.0 -1.0 79.553 0.0 -1.0 0.0 -10.175 1.0 -26.28 1.0 0.0 1 -true -13 1969-12-31 16:00:13.506 -13.0 1cGVWH7n1QU 13 -13 0 -13.0 -0.0 13.0 -1034.1889999999999 0.0 13.0 0.0 -10.175 -13.0 2.0215384615384617 -13.0 0.0 -13 -NULL 5 1969-12-31 16:00:04.058 5.0 NULL -5 5 0 5.0 0.0 -5.0 397.765 0.0 -5.0 0.0 -10.175 5.0 -5.256 5.0 0.0 5 -true -26 1969-12-31 16:00:01.542 -26.0 821UdmGbkEf4j 26 -26 0 -26.0 -0.0 26.0 -2068.3779999999997 0.0 26.0 0.0 -10.175 -26.0 1.0107692307692309 -26.0 0.0 -26 -NULL -23 1969-12-31 16:00:01.764 -23.0 NULL 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -NULL 57 1969-12-31 16:00:15.76 57.0 NULL -57 57 0 57.0 0.0 -57.0 4534.521 0.0 -57.0 0.0 -10.175 57.0 -0.4610526315789474 57.0 0.0 57 -NULL 9 1969-12-31 16:00:10.473 9.0 NULL -9 9 0 9.0 0.0 -9.0 715.977 0.0 -9.0 0.0 -10.175 9.0 -2.92 9.0 0.0 9 -true -10 1969-12-31 16:00:09.433 -10.0 821UdmGbkEf4j 10 -10 0 -10.0 -0.0 10.0 -795.53 0.0 10.0 0.0 -10.175 -10.0 2.628 -10.0 0.0 -10 -NULL 45 1969-12-31 16:00:07.497 45.0 NULL -45 45 0 45.0 0.0 -45.0 3579.8849999999998 0.0 -45.0 0.0 -10.175 45.0 -0.5840000000000001 45.0 0.0 45 -NULL -41 1969-12-31 16:00:14.973 -41.0 NULL 41 -41 0 -41.0 -0.0 41.0 -3261.673 0.0 41.0 0.0 -10.175 -41.0 0.6409756097560976 -41.0 0.0 -41 -true 46 1969-12-31 16:00:01.276 46.0 cvLH6Eat2yFsyy7p -46 46 0 46.0 0.0 -46.0 3659.438 0.0 -46.0 0.0 -10.175 46.0 -0.571304347826087 46.0 0.0 46 -NULL 3 1969-12-31 16:00:06.624 3.0 NULL -3 3 0 3.0 0.0 -3.0 238.659 0.0 -3.0 0.0 -10.175 3.0 -8.76 3.0 0.0 3 -NULL -46 1969-12-31 16:00:07.744 -46.0 NULL 46 -46 0 -46.0 -0.0 46.0 -3659.438 0.0 46.0 0.0 -10.175 -46.0 0.571304347826087 -46.0 0.0 -46 -NULL 26 1969-12-31 16:00:03.657 26.0 NULL -26 26 0 26.0 0.0 -26.0 2068.3779999999997 0.0 -26.0 0.0 -10.175 26.0 -1.0107692307692309 26.0 0.0 26 -true 0 1969-12-31 16:00:01.268 0.0 cvLH6Eat2yFsyy7p 0 0 0 0.0 0.0 -0.0 0.0 0.0 -0.0 0.0 -10.175 0.0 NULL 0.0 NULL 0 -NULL 2 1969-12-31 16:00:11.342 2.0 NULL -2 2 0 2.0 0.0 -2.0 159.106 0.0 -2.0 0.0 -10.175 2.0 -13.14 2.0 0.0 2 -true -24 1969-12-31 16:00:13.56 -24.0 1cGVWH7n1QU 24 -24 0 -24.0 -0.0 24.0 -1909.272 0.0 24.0 0.0 -10.175 -24.0 1.095 -24.0 0.0 -24 -true -55 1969-12-31 16:00:05.591 -55.0 1cGVWH7n1QU 55 -55 0 -55.0 -0.0 55.0 -4375.415 0.0 55.0 0.0 -10.175 -55.0 0.47781818181818186 -55.0 0.0 -55 -true 5 1969-12-31 16:00:09.238 5.0 821UdmGbkEf4j -5 5 0 5.0 0.0 -5.0 397.765 0.0 -5.0 0.0 -10.175 5.0 -5.256 5.0 0.0 5 -NULL -35 1969-12-31 16:00:13.819 -35.0 NULL 35 -35 0 -35.0 -0.0 35.0 -2784.355 0.0 35.0 0.0 -10.175 -35.0 0.7508571428571429 -35.0 0.0 -35 -NULL -64 1969-12-31 16:00:01.785 -64.0 NULL 64 -64 0 -64.0 -0.0 64.0 -5091.392 0.0 64.0 0.0 -10.175 -64.0 0.410625 -64.0 0.0 -64 -true -13 1969-12-31 16:00:15.588 -13.0 cvLH6Eat2yFsyy7p 13 -13 0 -13.0 -0.0 13.0 -1034.1889999999999 0.0 13.0 0.0 -10.175 -13.0 2.0215384615384617 -13.0 0.0 -13 -true -5 1969-12-31 16:00:00.037 -5.0 cvLH6Eat2yFsyy7p 5 -5 0 -5.0 -0.0 5.0 -397.765 0.0 5.0 0.0 -10.175 -5.0 5.256 -5.0 0.0 -5 -true -7 1969-12-31 16:00:00.315 -7.0 cvLH6Eat2yFsyy7p 7 -7 0 -7.0 -0.0 7.0 -556.871 0.0 7.0 0.0 -10.175 -7.0 3.7542857142857144 -7.0 0.0 -7 -NULL -16 1969-12-31 16:00:04.741 -16.0 NULL 16 -16 0 -16.0 -0.0 16.0 -1272.848 0.0 16.0 0.0 -10.175 -16.0 1.6425 -16.0 0.0 -16 -NULL -7 1969-12-31 16:00:08.319 -7.0 NULL 7 -7 0 -7.0 -0.0 7.0 -556.871 0.0 7.0 0.0 -10.175 -7.0 3.7542857142857144 -7.0 0.0 -7 -NULL -61 1969-12-31 16:00:05.613 -61.0 NULL 61 -61 0 -61.0 -0.0 61.0 -4852.733 0.0 61.0 0.0 -10.175 -61.0 0.4308196721311476 -61.0 0.0 -61 -NULL 31 1969-12-31 16:00:02.363 31.0 NULL -31 31 0 31.0 0.0 -31.0 2466.143 0.0 -31.0 0.0 -10.175 31.0 -0.847741935483871 31.0 0.0 31 -true -38 1969-12-31 16:00:11.319 -38.0 821UdmGbkEf4j 38 -38 0 -38.0 -0.0 38.0 -3023.014 0.0 38.0 0.0 -10.175 -38.0 0.6915789473684211 -38.0 0.0 -38 -true -36 1969-12-31 16:00:12.94 -36.0 821UdmGbkEf4j 36 -36 0 -36.0 -0.0 36.0 -2863.908 0.0 36.0 0.0 -10.175 -36.0 0.73 -36.0 0.0 -36 -NULL 39 1969-12-31 16:00:12.557 39.0 NULL -39 39 0 39.0 0.0 -39.0 3102.567 0.0 -39.0 0.0 -10.175 39.0 -0.6738461538461539 39.0 0.0 39 -NULL 29 1969-12-31 16:00:02.6 29.0 NULL -29 29 0 29.0 0.0 -29.0 2307.037 0.0 -29.0 0.0 -10.175 29.0 -0.9062068965517242 29.0 0.0 29 -NULL -19 1969-12-31 16:00:05.105 -19.0 NULL 19 -19 0 -19.0 -0.0 19.0 -1511.507 0.0 19.0 0.0 -10.175 -19.0 1.3831578947368421 -19.0 0.0 -19 -NULL 0 1969-12-31 16:00:02.71 0.0 NULL 0 0 0 0.0 0.0 -0.0 0.0 0.0 -0.0 0.0 -10.175 0.0 NULL 0.0 NULL 0 -NULL -4 1969-12-31 16:00:14.04 -4.0 NULL 4 -4 0 -4.0 -0.0 4.0 -318.212 0.0 4.0 0.0 -10.175 -4.0 6.57 -4.0 0.0 -4 -NULL 15 1969-12-31 16:00:00.515 15.0 NULL -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -true -49 1969-12-31 16:00:09.767 -49.0 cvLH6Eat2yFsyy7p 49 -49 0 -49.0 -0.0 49.0 -3898.0969999999998 0.0 49.0 0.0 -10.175 -49.0 0.5363265306122449 -49.0 0.0 -49 -NULL -21 1969-12-31 16:00:07.982 -21.0 NULL 21 -21 0 -21.0 -0.0 21.0 -1670.6129999999998 0.0 21.0 0.0 -10.175 -21.0 1.2514285714285716 -21.0 0.0 -21 -true -53 1969-12-31 16:00:11.419 -53.0 cvLH6Eat2yFsyy7p 53 -53 0 -53.0 -0.0 53.0 -4216.309 0.0 53.0 0.0 -10.175 -53.0 0.4958490566037736 -53.0 0.0 -53 -true 0 1969-12-31 16:00:05.166 0.0 1cGVWH7n1QU 0 0 0 0.0 0.0 -0.0 0.0 0.0 -0.0 0.0 -10.175 0.0 NULL 0.0 NULL 0 -true 27 1969-12-31 16:00:04.963 27.0 cvLH6Eat2yFsyy7p -27 27 0 27.0 0.0 -27.0 2147.931 0.0 -27.0 0.0 -10.175 27.0 -0.9733333333333334 27.0 0.0 27 -true -35 1969-12-31 16:00:10.321 -35.0 cvLH6Eat2yFsyy7p 35 -35 0 -35.0 -0.0 35.0 -2784.355 0.0 35.0 0.0 -10.175 -35.0 0.7508571428571429 -35.0 0.0 -35 -NULL -44 1969-12-31 16:00:03.333 -44.0 NULL 44 -44 0 -44.0 -0.0 44.0 -3500.332 0.0 44.0 0.0 -10.175 -44.0 0.5972727272727273 -44.0 0.0 -44 -NULL -4 1969-12-31 16:00:00.648 -4.0 NULL 4 -4 0 -4.0 -0.0 4.0 -318.212 0.0 4.0 0.0 -10.175 -4.0 6.57 -4.0 0.0 -4 -true 16 1969-12-31 16:00:08.735 16.0 1cGVWH7n1QU -16 16 0 16.0 0.0 -16.0 1272.848 0.0 -16.0 0.0 -10.175 16.0 -1.6425 16.0 0.0 16 -NULL 52 1969-12-31 16:00:04.024 52.0 NULL -52 52 0 52.0 0.0 -52.0 4136.755999999999 0.0 -52.0 0.0 -10.175 52.0 -0.5053846153846154 52.0 0.0 52 -true -36 1969-12-31 16:00:01.444 -36.0 cvLH6Eat2yFsyy7p 36 -36 0 -36.0 -0.0 36.0 -2863.908 0.0 36.0 0.0 -10.175 -36.0 0.73 -36.0 0.0 -36 -NULL 44 1969-12-31 16:00:06.91 44.0 NULL -44 44 0 44.0 0.0 -44.0 3500.332 0.0 -44.0 0.0 -10.175 44.0 -0.5972727272727273 44.0 0.0 44 -NULL -62 1969-12-31 16:00:09.025 -62.0 NULL 62 -62 0 -62.0 -0.0 62.0 -4932.286 0.0 62.0 0.0 -10.175 -62.0 0.4238709677419355 -62.0 0.0 -62 -true -34 1969-12-31 16:00:04.261 -34.0 821UdmGbkEf4j 34 -34 0 -34.0 -0.0 34.0 -2704.8019999999997 0.0 34.0 0.0 -10.175 -34.0 0.7729411764705882 -34.0 0.0 -34 -NULL -62 1969-12-31 16:00:14.517 -62.0 NULL 62 -62 0 -62.0 -0.0 62.0 -4932.286 0.0 62.0 0.0 -10.175 -62.0 0.4238709677419355 -62.0 0.0 -62 -NULL -39 1969-12-31 16:00:01.534 -39.0 NULL 39 -39 0 -39.0 -0.0 39.0 -3102.567 0.0 39.0 0.0 -10.175 -39.0 0.6738461538461539 -39.0 0.0 -39 -true -6 1969-12-31 16:00:01.845 -6.0 1cGVWH7n1QU 6 -6 0 -6.0 -0.0 6.0 -477.318 0.0 6.0 0.0 -10.175 -6.0 4.38 -6.0 0.0 -6 -NULL 32 1969-12-31 16:00:08 32.0 NULL -32 32 0 32.0 0.0 -32.0 2545.696 0.0 -32.0 0.0 -10.175 32.0 -0.82125 32.0 0.0 32 -true -6 1969-12-31 16:00:02.834 -6.0 821UdmGbkEf4j 6 -6 0 -6.0 -0.0 6.0 -477.318 0.0 6.0 0.0 -10.175 -6.0 4.38 -6.0 0.0 -6 -NULL -26 1969-12-31 16:00:05.037 -26.0 NULL 26 -26 0 -26.0 -0.0 26.0 -2068.3779999999997 0.0 26.0 0.0 -10.175 -26.0 1.0107692307692309 -26.0 0.0 -26 -NULL 45 1969-12-31 16:00:03.875 45.0 NULL -45 45 0 45.0 0.0 -45.0 3579.8849999999998 0.0 -45.0 0.0 -10.175 45.0 -0.5840000000000001 45.0 0.0 45 -true -43 1969-12-31 16:00:01.52 -43.0 1cGVWH7n1QU 43 -43 0 -43.0 -0.0 43.0 -3420.779 0.0 43.0 0.0 -10.175 -43.0 0.6111627906976744 -43.0 0.0 -43 -NULL -50 1969-12-31 16:00:08.638 -50.0 NULL 50 -50 0 -50.0 -0.0 50.0 -3977.6499999999996 0.0 50.0 0.0 -10.175 -50.0 0.5256000000000001 -50.0 0.0 -50 -NULL 22 1969-12-31 16:00:12.112 22.0 NULL -22 22 0 22.0 0.0 -22.0 1750.166 0.0 -22.0 0.0 -10.175 22.0 -1.1945454545454546 22.0 0.0 22 -NULL 50 1969-12-31 16:00:09.364 50.0 NULL -50 50 0 50.0 0.0 -50.0 3977.6499999999996 0.0 -50.0 0.0 -10.175 50.0 -0.5256000000000001 50.0 0.0 50 -NULL -15 1969-12-31 16:00:00.819 -15.0 NULL 15 -15 0 -15.0 -0.0 15.0 -1193.295 0.0 15.0 0.0 -10.175 -15.0 1.752 -15.0 0.0 -15 -NULL 52 1969-12-31 16:00:05.484 52.0 NULL -52 52 0 52.0 0.0 -52.0 4136.755999999999 0.0 -52.0 0.0 -10.175 52.0 -0.5053846153846154 52.0 0.0 52 -true 50 1969-12-31 16:00:09.041 50.0 1cGVWH7n1QU -50 50 0 50.0 0.0 -50.0 3977.6499999999996 0.0 -50.0 0.0 -10.175 50.0 -0.5256000000000001 50.0 0.0 50 -NULL -21 1969-12-31 16:00:01.599 -21.0 NULL 21 -21 0 -21.0 -0.0 21.0 -1670.6129999999998 0.0 21.0 0.0 -10.175 -21.0 1.2514285714285716 -21.0 0.0 -21 -NULL 28 1969-12-31 16:00:08.292 28.0 NULL -28 28 0 28.0 0.0 -28.0 2227.484 0.0 -28.0 0.0 -10.175 28.0 -0.9385714285714286 28.0 0.0 28 -NULL -8 1969-12-31 16:00:00.122 -8.0 NULL 8 -8 0 -8.0 -0.0 8.0 -636.424 0.0 8.0 0.0 -10.175 -8.0 3.285 -8.0 0.0 -8 -true 33 1969-12-31 16:00:09.924 33.0 cvLH6Eat2yFsyy7p -33 33 0 33.0 0.0 -33.0 2625.249 0.0 -33.0 0.0 -10.175 33.0 -0.7963636363636364 33.0 0.0 33 -NULL -16 1969-12-31 16:00:00.545 -16.0 NULL 16 -16 0 -16.0 -0.0 16.0 -1272.848 0.0 16.0 0.0 -10.175 -16.0 1.6425 -16.0 0.0 -16 -true -30 1969-12-31 16:00:10.003 -30.0 1cGVWH7n1QU 30 -30 0 -30.0 -0.0 30.0 -2386.59 0.0 30.0 0.0 -10.175 -30.0 0.876 -30.0 0.0 -30 -true 1 1969-12-31 16:00:00.106 1.0 1cGVWH7n1QU -1 1 0 1.0 0.0 -1.0 79.553 0.0 -1.0 0.0 -10.175 1.0 -26.28 1.0 0.0 1 -true -42 1969-12-31 16:00:01.783 -42.0 1cGVWH7n1QU 42 -42 0 -42.0 -0.0 42.0 -3341.2259999999997 0.0 42.0 0.0 -10.175 -42.0 0.6257142857142858 -42.0 0.0 -42 -NULL -41 1969-12-31 16:00:03.544 -41.0 NULL 41 -41 0 -41.0 -0.0 41.0 -3261.673 0.0 41.0 0.0 -10.175 -41.0 0.6409756097560976 -41.0 0.0 -41 -true -50 1969-12-31 16:00:11.234 -50.0 1cGVWH7n1QU 50 -50 0 -50.0 -0.0 50.0 -3977.6499999999996 0.0 50.0 0.0 -10.175 -50.0 0.5256000000000001 -50.0 0.0 -50 -true -56 1969-12-31 16:00:00.274 -56.0 821UdmGbkEf4j 56 -56 0 -56.0 -0.0 56.0 -4454.968 0.0 56.0 0.0 -10.175 -56.0 0.4692857142857143 -56.0 0.0 -56 -NULL -39 1969-12-31 16:00:13.358 -39.0 NULL 39 -39 0 -39.0 -0.0 39.0 -3102.567 0.0 39.0 0.0 -10.175 -39.0 0.6738461538461539 -39.0 0.0 -39 -NULL 33 1969-12-31 16:00:02.585 33.0 NULL -33 33 0 33.0 0.0 -33.0 2625.249 0.0 -33.0 0.0 -10.175 33.0 -0.7963636363636364 33.0 0.0 33 -true -9 1969-12-31 16:00:14.605 -9.0 821UdmGbkEf4j 9 -9 0 -9.0 -0.0 9.0 -715.977 0.0 9.0 0.0 -10.175 -9.0 2.92 -9.0 0.0 -9 -true 5 1969-12-31 16:00:11.803 5.0 1cGVWH7n1QU -5 5 0 5.0 0.0 -5.0 397.765 0.0 -5.0 0.0 -10.175 5.0 -5.256 5.0 0.0 5 -NULL 28 1969-12-31 16:00:10.674 28.0 NULL -28 28 0 28.0 0.0 -28.0 2227.484 0.0 -28.0 0.0 -10.175 28.0 -0.9385714285714286 28.0 0.0 28 -NULL 6 1969-12-31 16:00:09.111 6.0 NULL -6 6 0 6.0 0.0 -6.0 477.318 0.0 -6.0 0.0 -10.175 6.0 -4.38 6.0 0.0 6 -true 39 1969-12-31 16:00:06.228 39.0 cvLH6Eat2yFsyy7p -39 39 0 39.0 0.0 -39.0 3102.567 0.0 -39.0 0.0 -10.175 39.0 -0.6738461538461539 39.0 0.0 39 -NULL -61 1969-12-31 16:00:02.698 -61.0 NULL 61 -61 0 -61.0 -0.0 61.0 -4852.733 0.0 61.0 0.0 -10.175 -61.0 0.4308196721311476 -61.0 0.0 -61 -NULL 20 1969-12-31 16:00:31.808 20.0 NULL -20 20 0 20.0 0.0 -20.0 1591.06 0.0 -20.0 0.0 -10.175 20.0 -1.314 20.0 0.0 20 -NULL -49 1969-12-31 16:00:04.868 -49.0 NULL 49 -49 0 -49.0 -0.0 49.0 -3898.0969999999998 0.0 49.0 0.0 -10.175 -49.0 0.5363265306122449 -49.0 0.0 -49 -NULL -60 1969-12-31 16:00:11.641 -60.0 NULL 60 -60 0 -60.0 -0.0 60.0 -4773.18 0.0 60.0 0.0 -10.175 -60.0 0.438 -60.0 0.0 -60 -true 46 1969-12-31 16:00:04.301 46.0 1cGVWH7n1QU -46 46 0 46.0 0.0 -46.0 3659.438 0.0 -46.0 0.0 -10.175 46.0 -0.571304347826087 46.0 0.0 46 -NULL 45 1969-12-31 16:00:12.295 45.0 NULL -45 45 0 45.0 0.0 -45.0 3579.8849999999998 0.0 -45.0 0.0 -10.175 45.0 -0.5840000000000001 45.0 0.0 45 -NULL 40 1969-12-31 16:00:07.109 40.0 NULL -40 40 0 40.0 0.0 -40.0 3182.12 0.0 -40.0 0.0 -10.175 40.0 -0.657 40.0 0.0 40 -NULL -55 1969-12-31 16:00:08.896 -55.0 NULL 55 -55 0 -55.0 -0.0 55.0 -4375.415 0.0 55.0 0.0 -10.175 -55.0 0.47781818181818186 -55.0 0.0 -55 -NULL 46 1969-12-31 16:00:01.388 46.0 NULL -46 46 0 46.0 0.0 -46.0 3659.438 0.0 -46.0 0.0 -10.175 46.0 -0.571304347826087 46.0 0.0 46 -NULL -7 1969-12-31 16:00:00.523 -7.0 NULL 7 -7 0 -7.0 -0.0 7.0 -556.871 0.0 7.0 0.0 -10.175 -7.0 3.7542857142857144 -7.0 0.0 -7 -NULL -33 1969-12-31 16:00:10.959 -33.0 NULL 33 -33 0 -33.0 -0.0 33.0 -2625.249 0.0 33.0 0.0 -10.175 -33.0 0.7963636363636364 -33.0 0.0 -33 -NULL 54 1969-12-31 16:00:08.241 54.0 NULL -54 54 0 54.0 0.0 -54.0 4295.862 0.0 -54.0 0.0 -10.175 54.0 -0.4866666666666667 54.0 0.0 54 -true -57 1969-12-31 16:00:00.56 -57.0 1cGVWH7n1QU 57 -57 0 -57.0 -0.0 57.0 -4534.521 0.0 57.0 0.0 -10.175 -57.0 0.4610526315789474 -57.0 0.0 -57 -NULL 40 1969-12-31 16:00:06.346 40.0 NULL -40 40 0 40.0 0.0 -40.0 3182.12 0.0 -40.0 0.0 -10.175 40.0 -0.657 40.0 0.0 40 -true 35 1969-12-31 16:00:00.508 35.0 cvLH6Eat2yFsyy7p -35 35 0 35.0 0.0 -35.0 2784.355 0.0 -35.0 0.0 -10.175 35.0 -0.7508571428571429 35.0 0.0 35 -true -58 1969-12-31 16:00:07.288 -58.0 821UdmGbkEf4j 58 -58 0 -58.0 -0.0 58.0 -4614.074 0.0 58.0 0.0 -10.175 -58.0 0.4531034482758621 -58.0 0.0 -58 -true 4 1969-12-31 16:00:09.934 4.0 1cGVWH7n1QU -4 4 0 4.0 0.0 -4.0 318.212 0.0 -4.0 0.0 -10.175 4.0 -6.57 4.0 0.0 4 -NULL -14 1969-12-31 16:00:04.574 -14.0 NULL 14 -14 0 -14.0 -0.0 14.0 -1113.742 0.0 14.0 0.0 -10.175 -14.0 1.8771428571428572 -14.0 0.0 -14 -true -40 1969-12-31 16:00:07.985 -40.0 821UdmGbkEf4j 40 -40 0 -40.0 -0.0 40.0 -3182.12 0.0 40.0 0.0 -10.175 -40.0 0.657 -40.0 0.0 -40 -NULL -36 1969-12-31 16:00:02.066 -36.0 NULL 36 -36 0 -36.0 -0.0 36.0 -2863.908 0.0 36.0 0.0 -10.175 -36.0 0.73 -36.0 0.0 -36 -true -20 1969-12-31 16:00:00.278 -20.0 821UdmGbkEf4j 20 -20 0 -20.0 -0.0 20.0 -1591.06 0.0 20.0 0.0 -10.175 -20.0 1.314 -20.0 0.0 -20 -NULL -7 1969-12-31 16:00:07.058 -7.0 NULL 7 -7 0 -7.0 -0.0 7.0 -556.871 0.0 7.0 0.0 -10.175 -7.0 3.7542857142857144 -7.0 0.0 -7 -true -13 1969-12-31 16:00:15.223 -13.0 1cGVWH7n1QU 13 -13 0 -13.0 -0.0 13.0 -1034.1889999999999 0.0 13.0 0.0 -10.175 -13.0 2.0215384615384617 -13.0 0.0 -13 -NULL -9 1969-12-31 16:00:14.094 -9.0 NULL 9 -9 0 -9.0 -0.0 9.0 -715.977 0.0 9.0 0.0 -10.175 -9.0 2.92 -9.0 0.0 -9 -NULL 49 1969-12-31 16:00:02.657 49.0 NULL -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 -true 11 1969-12-31 16:00:08.95 11.0 821UdmGbkEf4j -11 11 0 11.0 0.0 -11.0 875.083 0.0 -11.0 0.0 -10.175 11.0 -2.389090909090909 11.0 0.0 11 -true 51 1969-12-31 16:00:00.882 51.0 821UdmGbkEf4j -51 51 0 51.0 0.0 -51.0 4057.203 0.0 -51.0 0.0 -10.175 51.0 -0.5152941176470588 51.0 0.0 51 -true 26 1969-12-31 16:00:10.776 26.0 1cGVWH7n1QU -26 26 0 26.0 0.0 -26.0 2068.3779999999997 0.0 -26.0 0.0 -10.175 26.0 -1.0107692307692309 26.0 0.0 26 -true 0 1969-12-31 16:00:09.566 0.0 cvLH6Eat2yFsyy7p 0 0 0 0.0 0.0 -0.0 0.0 0.0 -0.0 0.0 -10.175 0.0 NULL 0.0 NULL 0 -NULL -41 1969-12-31 16:00:05.609 -41.0 NULL 41 -41 0 -41.0 -0.0 41.0 -3261.673 0.0 41.0 0.0 -10.175 -41.0 0.6409756097560976 -41.0 0.0 -41 -true -48 1969-12-31 16:00:02.316 -48.0 cvLH6Eat2yFsyy7p 48 -48 0 -48.0 -0.0 48.0 -3818.544 0.0 48.0 0.0 -10.175 -48.0 0.5475 -48.0 0.0 -48 -NULL -48 1969-12-31 16:00:07.046 -48.0 NULL 48 -48 0 -48.0 -0.0 48.0 -3818.544 0.0 48.0 0.0 -10.175 -48.0 0.5475 -48.0 0.0 -48 -NULL -10 1969-12-31 16:00:06.413 -10.0 NULL 10 -10 0 -10.0 -0.0 10.0 -795.53 0.0 10.0 0.0 -10.175 -10.0 2.628 -10.0 0.0 -10 -true 60 1969-12-31 16:00:02.089 60.0 1cGVWH7n1QU -60 60 0 60.0 0.0 -60.0 4773.18 0.0 -60.0 0.0 -10.175 60.0 -0.438 60.0 0.0 60 -NULL -16 1969-12-31 16:00:02.267 -16.0 NULL 16 -16 0 -16.0 -0.0 16.0 -1272.848 0.0 16.0 0.0 -10.175 -16.0 1.6425 -16.0 0.0 -16 -NULL -45 1969-12-31 16:00:02.814 -45.0 NULL 45 -45 0 -45.0 -0.0 45.0 -3579.8849999999998 0.0 45.0 0.0 -10.175 -45.0 0.5840000000000001 -45.0 0.0 -45 -NULL -9 1969-12-31 16:00:10.92 -9.0 NULL 9 -9 0 -9.0 -0.0 9.0 -715.977 0.0 9.0 0.0 -10.175 -9.0 2.92 -9.0 0.0 -9 -NULL 62 1969-12-31 16:00:07.828 62.0 NULL -62 62 0 62.0 0.0 -62.0 4932.286 0.0 -62.0 0.0 -10.175 62.0 -0.4238709677419355 62.0 0.0 62 -true -19 1969-12-31 16:00:01.577 -19.0 821UdmGbkEf4j 19 -19 0 -19.0 -0.0 19.0 -1511.507 0.0 19.0 0.0 -10.175 -19.0 1.3831578947368421 -19.0 0.0 -19 -true -1 1969-12-31 16:00:15.065 -1.0 821UdmGbkEf4j 1 -1 0 -1.0 -0.0 1.0 -79.553 0.0 1.0 0.0 -10.175 -1.0 26.28 -1.0 0.0 -1 -true -36 1969-12-31 16:00:09.972 -36.0 1cGVWH7n1QU 36 -36 0 -36.0 -0.0 36.0 -2863.908 0.0 36.0 0.0 -10.175 -36.0 0.73 -36.0 0.0 -36 -true -58 1969-12-31 16:00:12.918 -58.0 cvLH6Eat2yFsyy7p 58 -58 0 -58.0 -0.0 58.0 -4614.074 0.0 58.0 0.0 -10.175 -58.0 0.4531034482758621 -58.0 0.0 -58 -NULL -18 1969-12-31 16:00:04.071 -18.0 NULL 18 -18 0 -18.0 -0.0 18.0 -1431.954 0.0 18.0 0.0 -10.175 -18.0 1.46 -18.0 0.0 -18 -true -51 1969-12-31 16:00:04.159 -51.0 cvLH6Eat2yFsyy7p 51 -51 0 -51.0 -0.0 51.0 -4057.203 0.0 51.0 0.0 -10.175 -51.0 0.5152941176470588 -51.0 0.0 -51 -NULL -57 1969-12-31 16:00:02.707 -57.0 NULL 57 -57 0 -57.0 -0.0 57.0 -4534.521 0.0 57.0 0.0 -10.175 -57.0 0.4610526315789474 -57.0 0.0 -57 -true -31 1969-12-31 16:00:09.452 -31.0 821UdmGbkEf4j 31 -31 0 -31.0 -0.0 31.0 -2466.143 0.0 31.0 0.0 -10.175 -31.0 0.847741935483871 -31.0 0.0 -31 -true -6 1969-12-31 16:00:06.209 -6.0 1cGVWH7n1QU 6 -6 0 -6.0 -0.0 6.0 -477.318 0.0 6.0 0.0 -10.175 -6.0 4.38 -6.0 0.0 -6 -true -29 1969-12-31 16:00:08.488 -29.0 1cGVWH7n1QU 29 -29 0 -29.0 -0.0 29.0 -2307.037 0.0 29.0 0.0 -10.175 -29.0 0.9062068965517242 -29.0 0.0 -29 -true 33 1969-12-31 16:00:11.673 33.0 821UdmGbkEf4j -33 33 0 33.0 0.0 -33.0 2625.249 0.0 -33.0 0.0 -10.175 33.0 -0.7963636363636364 33.0 0.0 33 -true 53 1969-12-31 16:00:09.926 53.0 821UdmGbkEf4j -53 53 0 53.0 0.0 -53.0 4216.309 0.0 -53.0 0.0 -10.175 53.0 -0.4958490566037736 53.0 0.0 53 -true -44 1969-12-31 16:00:03.152 -44.0 821UdmGbkEf4j 44 -44 0 -44.0 -0.0 44.0 -3500.332 0.0 44.0 0.0 -10.175 -44.0 0.5972727272727273 -44.0 0.0 -44 -NULL -40 1969-12-31 16:00:05.192 -40.0 NULL 40 -40 0 -40.0 -0.0 40.0 -3182.12 0.0 40.0 0.0 -10.175 -40.0 0.657 -40.0 0.0 -40 -NULL -12 1969-12-31 16:00:01.059 -12.0 NULL 12 -12 0 -12.0 -0.0 12.0 -954.636 0.0 12.0 0.0 -10.175 -12.0 2.19 -12.0 0.0 -12 -true -6 1969-12-31 16:00:03.613 -6.0 821UdmGbkEf4j 6 -6 0 -6.0 -0.0 6.0 -477.318 0.0 6.0 0.0 -10.175 -6.0 4.38 -6.0 0.0 -6 -true 18 1969-12-31 16:00:03.108 18.0 1cGVWH7n1QU -18 18 0 18.0 0.0 -18.0 1431.954 0.0 -18.0 0.0 -10.175 18.0 -1.46 18.0 0.0 18 -NULL -28 1969-12-31 16:00:00.326 -28.0 NULL 28 -28 0 -28.0 -0.0 28.0 -2227.484 0.0 28.0 0.0 -10.175 -28.0 0.9385714285714286 -28.0 0.0 -28 -true 43 1969-12-31 16:00:03.818 43.0 1cGVWH7n1QU -43 43 0 43.0 0.0 -43.0 3420.779 0.0 -43.0 0.0 -10.175 43.0 -0.6111627906976744 43.0 0.0 43 -true 21 1969-12-31 16:00:03.882 21.0 1cGVWH7n1QU -21 21 0 21.0 0.0 -21.0 1670.6129999999998 0.0 -21.0 0.0 -10.175 21.0 -1.2514285714285716 21.0 0.0 21 -true -47 1969-12-31 16:00:00.984 -47.0 1cGVWH7n1QU 47 -47 0 -47.0 -0.0 47.0 -3738.991 0.0 47.0 0.0 -10.175 -47.0 0.5591489361702128 -47.0 0.0 -47 -NULL -58 1969-12-31 16:00:04.682 -58.0 NULL 58 -58 0 -58.0 -0.0 58.0 -4614.074 0.0 58.0 0.0 -10.175 -58.0 0.4531034482758621 -58.0 0.0 -58 -true 37 1969-12-31 16:00:04.495 37.0 cvLH6Eat2yFsyy7p -37 37 0 37.0 0.0 -37.0 2943.461 0.0 -37.0 0.0 -10.175 37.0 -0.7102702702702703 37.0 0.0 37 -NULL 62 1969-12-31 16:00:00.056 62.0 NULL -62 62 0 62.0 0.0 -62.0 4932.286 0.0 -62.0 0.0 -10.175 62.0 -0.4238709677419355 62.0 0.0 62 -true 60 1969-12-31 16:00:05.963 60.0 1cGVWH7n1QU -60 60 0 60.0 0.0 -60.0 4773.18 0.0 -60.0 0.0 -10.175 60.0 -0.438 60.0 0.0 60 -true 1 1969-12-31 16:00:11.583 1.0 821UdmGbkEf4j -1 1 0 1.0 0.0 -1.0 79.553 0.0 -1.0 0.0 -10.175 1.0 -26.28 1.0 0.0 1 -NULL 40 1969-12-31 16:00:01.356 40.0 NULL -40 40 0 40.0 0.0 -40.0 3182.12 0.0 -40.0 0.0 -10.175 40.0 -0.657 40.0 0.0 40 -true -25 1969-12-31 16:00:15.516 -25.0 821UdmGbkEf4j 25 -25 0 -25.0 -0.0 25.0 -1988.8249999999998 0.0 25.0 0.0 -10.175 -25.0 1.0512000000000001 -25.0 0.0 -25 -true 62 1969-12-31 16:00:01.674 62.0 cvLH6Eat2yFsyy7p -62 62 0 62.0 0.0 -62.0 4932.286 0.0 -62.0 0.0 -10.175 62.0 -0.4238709677419355 62.0 0.0 62 -true -52 1969-12-31 16:00:01.455 -52.0 cvLH6Eat2yFsyy7p 52 -52 0 -52.0 -0.0 52.0 -4136.755999999999 0.0 52.0 0.0 -10.175 -52.0 0.5053846153846154 -52.0 0.0 -52 -NULL -28 1969-12-31 16:00:00.003 -28.0 NULL 28 -28 0 -28.0 -0.0 28.0 -2227.484 0.0 28.0 0.0 -10.175 -28.0 0.9385714285714286 -28.0 0.0 -28 -NULL 61 1969-12-31 16:00:11.132 61.0 NULL -61 61 0 61.0 0.0 -61.0 4852.733 0.0 -61.0 0.0 -10.175 61.0 -0.4308196721311476 61.0 0.0 61 -true -55 1969-12-31 16:00:05.36 -55.0 1cGVWH7n1QU 55 -55 0 -55.0 -0.0 55.0 -4375.415 0.0 55.0 0.0 -10.175 -55.0 0.47781818181818186 -55.0 0.0 -55 -true -59 1969-12-31 16:00:00.516 -59.0 cvLH6Eat2yFsyy7p 59 -59 0 -59.0 -0.0 59.0 -4693.6269999999995 0.0 59.0 0.0 -10.175 -59.0 0.44542372881355935 -59.0 0.0 -59 -NULL -51 1969-12-31 16:00:12.141 -51.0 NULL 51 -51 0 -51.0 -0.0 51.0 -4057.203 0.0 51.0 0.0 -10.175 -51.0 0.5152941176470588 -51.0 0.0 -51 -true -39 1969-12-31 16:00:03.694 -39.0 821UdmGbkEf4j 39 -39 0 -39.0 -0.0 39.0 -3102.567 0.0 39.0 0.0 -10.175 -39.0 0.6738461538461539 -39.0 0.0 -39 -NULL 49 1969-12-31 16:00:13.503 49.0 NULL -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 -true -15 1969-12-31 16:00:01.343 -15.0 821UdmGbkEf4j 15 -15 0 -15.0 -0.0 15.0 -1193.295 0.0 15.0 0.0 -10.175 -15.0 1.752 -15.0 0.0 -15 -NULL -38 1969-12-31 16:00:05.324 -38.0 NULL 38 -38 0 -38.0 -0.0 38.0 -3023.014 0.0 38.0 0.0 -10.175 -38.0 0.6915789473684211 -38.0 0.0 -38 -true -33 1969-12-31 16:00:07.952 -33.0 cvLH6Eat2yFsyy7p 33 -33 0 -33.0 -0.0 33.0 -2625.249 0.0 33.0 0.0 -10.175 -33.0 0.7963636363636364 -33.0 0.0 -33 -NULL 41 1969-12-31 16:00:10.42 41.0 NULL -41 41 0 41.0 0.0 -41.0 3261.673 0.0 -41.0 0.0 -10.175 41.0 -0.6409756097560976 41.0 0.0 41 -true -49 1969-12-31 16:00:10.242 -49.0 821UdmGbkEf4j 49 -49 0 -49.0 -0.0 49.0 -3898.0969999999998 0.0 49.0 0.0 -10.175 -49.0 0.5363265306122449 -49.0 0.0 -49 -true 30 1969-12-31 16:00:05.847 30.0 cvLH6Eat2yFsyy7p -30 30 0 30.0 0.0 -30.0 2386.59 0.0 -30.0 0.0 -10.175 30.0 -0.876 30.0 0.0 30 -NULL 30 1969-12-31 16:00:03.809 30.0 NULL -30 30 0 30.0 0.0 -30.0 2386.59 0.0 -30.0 0.0 -10.175 30.0 -0.876 30.0 0.0 30 -true -36 1969-12-31 16:00:09.433 -36.0 cvLH6Eat2yFsyy7p 36 -36 0 -36.0 -0.0 36.0 -2863.908 0.0 36.0 0.0 -10.175 -36.0 0.73 -36.0 0.0 -36 -NULL -8 1969-12-31 16:00:00.054 -8.0 NULL 8 -8 0 -8.0 -0.0 8.0 -636.424 0.0 8.0 0.0 -10.175 -8.0 3.285 -8.0 0.0 -8 -true -34 1969-12-31 16:00:14.043 -34.0 821UdmGbkEf4j 34 -34 0 -34.0 -0.0 34.0 -2704.8019999999997 0.0 34.0 0.0 -10.175 -34.0 0.7729411764705882 -34.0 0.0 -34 -true 7 1969-12-31 16:00:07.336 7.0 1cGVWH7n1QU -7 7 0 7.0 0.0 -7.0 556.871 0.0 -7.0 0.0 -10.175 7.0 -3.7542857142857144 7.0 0.0 7 -true 15 1969-12-31 16:00:12.078 15.0 cvLH6Eat2yFsyy7p -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -NULL 38 1969-12-31 16:00:00.887 38.0 NULL -38 38 0 38.0 0.0 -38.0 3023.014 0.0 -38.0 0.0 -10.175 38.0 -0.6915789473684211 38.0 0.0 38 -true -20 1969-12-31 16:00:13.8 -20.0 cvLH6Eat2yFsyy7p 20 -20 0 -20.0 -0.0 20.0 -1591.06 0.0 20.0 0.0 -10.175 -20.0 1.314 -20.0 0.0 -20 -true 47 1969-12-31 16:00:11.112 47.0 821UdmGbkEf4j -47 47 0 47.0 0.0 -47.0 3738.991 0.0 -47.0 0.0 -10.175 47.0 -0.5591489361702128 47.0 0.0 47 -NULL -48 1969-12-31 16:00:00.96 -48.0 NULL 48 -48 0 -48.0 -0.0 48.0 -3818.544 0.0 48.0 0.0 -10.175 -48.0 0.5475 -48.0 0.0 -48 -NULL 34 1969-12-31 16:00:13.955 34.0 NULL -34 34 0 34.0 0.0 -34.0 2704.8019999999997 0.0 -34.0 0.0 -10.175 34.0 -0.7729411764705882 34.0 0.0 34 -true 19 1969-12-31 16:00:11.847 19.0 cvLH6Eat2yFsyy7p -19 19 0 19.0 0.0 -19.0 1511.507 0.0 -19.0 0.0 -10.175 19.0 -1.3831578947368421 19.0 0.0 19 -true 39 1969-12-31 16:00:10.399 39.0 1cGVWH7n1QU -39 39 0 39.0 0.0 -39.0 3102.567 0.0 -39.0 0.0 -10.175 39.0 -0.6738461538461539 39.0 0.0 39 -NULL -29 1969-12-31 16:00:00.699 -29.0 NULL 29 -29 0 -29.0 -0.0 29.0 -2307.037 0.0 29.0 0.0 -10.175 -29.0 0.9062068965517242 -29.0 0.0 -29 -NULL -57 1969-12-31 16:00:11.883 -57.0 NULL 57 -57 0 -57.0 -0.0 57.0 -4534.521 0.0 57.0 0.0 -10.175 -57.0 0.4610526315789474 -57.0 0.0 -57 -NULL 39 1969-12-31 16:00:15.146 39.0 NULL -39 39 0 39.0 0.0 -39.0 3102.567 0.0 -39.0 0.0 -10.175 39.0 -0.6738461538461539 39.0 0.0 39 -true 1 1969-12-31 16:00:00.125 1.0 cvLH6Eat2yFsyy7p -1 1 0 1.0 0.0 -1.0 79.553 0.0 -1.0 0.0 -10.175 1.0 -26.28 1.0 0.0 1 -NULL 13 1969-12-31 16:00:07.723 13.0 NULL -13 13 0 13.0 0.0 -13.0 1034.1889999999999 0.0 -13.0 0.0 -10.175 13.0 -2.0215384615384617 13.0 0.0 13 -true -52 1969-12-31 16:00:07.94 -52.0 cvLH6Eat2yFsyy7p 52 -52 0 -52.0 -0.0 52.0 -4136.755999999999 0.0 52.0 0.0 -10.175 -52.0 0.5053846153846154 -52.0 0.0 -52 -true 26 1969-12-31 16:00:02.407 26.0 cvLH6Eat2yFsyy7p -26 26 0 26.0 0.0 -26.0 2068.3779999999997 0.0 -26.0 0.0 -10.175 26.0 -1.0107692307692309 26.0 0.0 26 -NULL -47 1969-12-31 16:00:00.074 -47.0 NULL 47 -47 0 -47.0 -0.0 47.0 -3738.991 0.0 47.0 0.0 -10.175 -47.0 0.5591489361702128 -47.0 0.0 -47 -true 41 1969-12-31 16:00:00.085 41.0 1cGVWH7n1QU -41 41 0 41.0 0.0 -41.0 3261.673 0.0 -41.0 0.0 -10.175 41.0 -0.6409756097560976 41.0 0.0 41 -NULL -14 1969-12-31 16:00:16.274 -14.0 NULL 14 -14 0 -14.0 -0.0 14.0 -1113.742 0.0 14.0 0.0 -10.175 -14.0 1.8771428571428572 -14.0 0.0 -14 -NULL 25 1969-12-31 16:00:05.555 25.0 NULL -25 25 0 25.0 0.0 -25.0 1988.8249999999998 0.0 -25.0 0.0 -10.175 25.0 -1.0512000000000001 25.0 0.0 25 -NULL 35 1969-12-31 16:00:07.763 35.0 NULL -35 35 0 35.0 0.0 -35.0 2784.355 0.0 -35.0 0.0 -10.175 35.0 -0.7508571428571429 35.0 0.0 35 -NULL -33 1969-12-31 16:00:08.493 -33.0 NULL 33 -33 0 -33.0 -0.0 33.0 -2625.249 0.0 33.0 0.0 -10.175 -33.0 0.7963636363636364 -33.0 0.0 -33 -NULL 36 1969-12-31 16:00:15.446 36.0 NULL -36 36 0 36.0 0.0 -36.0 2863.908 0.0 -36.0 0.0 -10.175 36.0 -0.73 36.0 0.0 36 -NULL 59 1969-12-31 16:00:04.284 59.0 NULL -59 59 0 59.0 0.0 -59.0 4693.6269999999995 0.0 -59.0 0.0 -10.175 59.0 -0.44542372881355935 59.0 0.0 59 -true -52 1969-12-31 16:00:02.177 -52.0 1cGVWH7n1QU 52 -52 0 -52.0 -0.0 52.0 -4136.755999999999 0.0 52.0 0.0 -10.175 -52.0 0.5053846153846154 -52.0 0.0 -52 -true 24 1969-12-31 16:00:01.85 24.0 821UdmGbkEf4j -24 24 0 24.0 0.0 -24.0 1909.272 0.0 -24.0 0.0 -10.175 24.0 -1.095 24.0 0.0 24 -NULL 53 1969-12-31 16:00:15.223 53.0 NULL -53 53 0 53.0 0.0 -53.0 4216.309 0.0 -53.0 0.0 -10.175 53.0 -0.4958490566037736 53.0 0.0 53 -NULL 32 1969-12-31 16:00:00.12 32.0 NULL -32 32 0 32.0 0.0 -32.0 2545.696 0.0 -32.0 0.0 -10.175 32.0 -0.82125 32.0 0.0 32 -true 14 1969-12-31 16:00:12.103 14.0 1cGVWH7n1QU -14 14 0 14.0 0.0 -14.0 1113.742 0.0 -14.0 0.0 -10.175 14.0 -1.8771428571428572 14.0 0.0 14 -true -43 1969-12-31 16:00:01.416 -43.0 821UdmGbkEf4j 43 -43 0 -43.0 -0.0 43.0 -3420.779 0.0 43.0 0.0 -10.175 -43.0 0.6111627906976744 -43.0 0.0 -43 -NULL -43 1969-12-31 16:00:10.835 -43.0 NULL 43 -43 0 -43.0 -0.0 43.0 -3420.779 0.0 43.0 0.0 -10.175 -43.0 0.6111627906976744 -43.0 0.0 -43 -NULL -23 1969-12-31 16:00:00.244 -23.0 NULL 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -NULL 27 1969-12-31 16:00:00.381 27.0 NULL -27 27 0 27.0 0.0 -27.0 2147.931 0.0 -27.0 0.0 -10.175 27.0 -0.9733333333333334 27.0 0.0 27 -NULL 12 1969-12-31 16:00:12.772 12.0 NULL -12 12 0 12.0 0.0 -12.0 954.636 0.0 -12.0 0.0 -10.175 12.0 -2.19 12.0 0.0 12 -true 18 1969-12-31 16:00:10.881 18.0 1cGVWH7n1QU -18 18 0 18.0 0.0 -18.0 1431.954 0.0 -18.0 0.0 -10.175 18.0 -1.46 18.0 0.0 18 -true -47 1969-12-31 16:00:08.056 -47.0 1cGVWH7n1QU 47 -47 0 -47.0 -0.0 47.0 -3738.991 0.0 47.0 0.0 -10.175 -47.0 0.5591489361702128 -47.0 0.0 -47 -NULL 6 1969-12-31 16:00:07.713 6.0 NULL -6 6 0 6.0 0.0 -6.0 477.318 0.0 -6.0 0.0 -10.175 6.0 -4.38 6.0 0.0 6 -NULL 6 1969-12-31 16:00:05.688 6.0 NULL -6 6 0 6.0 0.0 -6.0 477.318 0.0 -6.0 0.0 -10.175 6.0 -4.38 6.0 0.0 6 -NULL 18 1969-12-31 16:00:02.659 18.0 NULL -18 18 0 18.0 0.0 -18.0 1431.954 0.0 -18.0 0.0 -10.175 18.0 -1.46 18.0 0.0 18 -NULL 41 1969-12-31 16:00:08.418 41.0 NULL -41 41 0 41.0 0.0 -41.0 3261.673 0.0 -41.0 0.0 -10.175 41.0 -0.6409756097560976 41.0 0.0 41 -true -11 1969-12-31 16:00:08.448 -11.0 821UdmGbkEf4j 11 -11 0 -11.0 -0.0 11.0 -875.083 0.0 11.0 0.0 -10.175 -11.0 2.389090909090909 -11.0 0.0 -11 -true 3 1969-12-31 16:00:01.634 3.0 cvLH6Eat2yFsyy7p -3 3 0 3.0 0.0 -3.0 238.659 0.0 -3.0 0.0 -10.175 3.0 -8.76 3.0 0.0 3 -true 22 1969-12-31 16:00:09.701 22.0 1cGVWH7n1QU -22 22 0 22.0 0.0 -22.0 1750.166 0.0 -22.0 0.0 -10.175 22.0 -1.1945454545454546 22.0 0.0 22 -true -58 1969-12-31 16:00:03.782 -58.0 cvLH6Eat2yFsyy7p 58 -58 0 -58.0 -0.0 58.0 -4614.074 0.0 58.0 0.0 -10.175 -58.0 0.4531034482758621 -58.0 0.0 -58 -NULL -5 1969-12-31 16:00:02.502 -5.0 NULL 5 -5 0 -5.0 -0.0 5.0 -397.765 0.0 5.0 0.0 -10.175 -5.0 5.256 -5.0 0.0 -5 -NULL 50 1969-12-31 16:00:00.022 50.0 NULL -50 50 0 50.0 0.0 -50.0 3977.6499999999996 0.0 -50.0 0.0 -10.175 50.0 -0.5256000000000001 50.0 0.0 50 -true -44 1969-12-31 16:00:04.357 -44.0 cvLH6Eat2yFsyy7p 44 -44 0 -44.0 -0.0 44.0 -3500.332 0.0 44.0 0.0 -10.175 -44.0 0.5972727272727273 -44.0 0.0 -44 -NULL -43 1969-12-31 16:00:16.096 -43.0 NULL 43 -43 0 -43.0 -0.0 43.0 -3420.779 0.0 43.0 0.0 -10.175 -43.0 0.6111627906976744 -43.0 0.0 -43 -NULL -9 1969-12-31 16:00:03.167 -9.0 NULL 9 -9 0 -9.0 -0.0 9.0 -715.977 0.0 9.0 0.0 -10.175 -9.0 2.92 -9.0 0.0 -9 -true 47 1969-12-31 16:00:05.241 47.0 cvLH6Eat2yFsyy7p -47 47 0 47.0 0.0 -47.0 3738.991 0.0 -47.0 0.0 -10.175 47.0 -0.5591489361702128 47.0 0.0 47 -true 4 1969-12-31 16:00:15.509 4.0 821UdmGbkEf4j -4 4 0 4.0 0.0 -4.0 318.212 0.0 -4.0 0.0 -10.175 4.0 -6.57 4.0 0.0 4 -true -1 1969-12-31 16:00:11.932 -1.0 cvLH6Eat2yFsyy7p 1 -1 0 -1.0 -0.0 1.0 -79.553 0.0 1.0 0.0 -10.175 -1.0 26.28 -1.0 0.0 -1 -true 16 1969-12-31 16:00:02.28 16.0 cvLH6Eat2yFsyy7p -16 16 0 16.0 0.0 -16.0 1272.848 0.0 -16.0 0.0 -10.175 16.0 -1.6425 16.0 0.0 16 -true -6 1969-12-31 16:00:12.157 -6.0 cvLH6Eat2yFsyy7p 6 -6 0 -6.0 -0.0 6.0 -477.318 0.0 6.0 0.0 -10.175 -6.0 4.38 -6.0 0.0 -6 -NULL -52 1969-12-31 16:00:03.963 -52.0 NULL 52 -52 0 -52.0 -0.0 52.0 -4136.755999999999 0.0 52.0 0.0 -10.175 -52.0 0.5053846153846154 -52.0 0.0 -52 -true -4 1969-12-31 16:00:08.5 -4.0 1cGVWH7n1QU 4 -4 0 -4.0 -0.0 4.0 -318.212 0.0 4.0 0.0 -10.175 -4.0 6.57 -4.0 0.0 -4 -true 50 1969-12-31 16:00:04.834 50.0 1cGVWH7n1QU -50 50 0 50.0 0.0 -50.0 3977.6499999999996 0.0 -50.0 0.0 -10.175 50.0 -0.5256000000000001 50.0 0.0 50 -NULL -63 1969-12-31 16:00:01.843 -63.0 NULL 63 -63 0 -63.0 -0.0 63.0 -5011.839 0.0 63.0 0.0 -10.175 -63.0 0.41714285714285715 -63.0 0.0 -63 -NULL 11 1969-12-31 16:00:09.984 11.0 NULL -11 11 0 11.0 0.0 -11.0 875.083 0.0 -11.0 0.0 -10.175 11.0 -2.389090909090909 11.0 0.0 11 -NULL -53 1969-12-31 16:00:11.746 -53.0 NULL 53 -53 0 -53.0 -0.0 53.0 -4216.309 0.0 53.0 0.0 -10.175 -53.0 0.4958490566037736 -53.0 0.0 -53 -true 22 1969-12-31 16:00:12.272 22.0 cvLH6Eat2yFsyy7p -22 22 0 22.0 0.0 -22.0 1750.166 0.0 -22.0 0.0 -10.175 22.0 -1.1945454545454546 22.0 0.0 22 -NULL 34 1969-12-31 16:00:02.7 34.0 NULL -34 34 0 34.0 0.0 -34.0 2704.8019999999997 0.0 -34.0 0.0 -10.175 34.0 -0.7729411764705882 34.0 0.0 34 -NULL -59 1969-12-31 16:00:06.777 -59.0 NULL 59 -59 0 -59.0 -0.0 59.0 -4693.6269999999995 0.0 59.0 0.0 -10.175 -59.0 0.44542372881355935 -59.0 0.0 -59 -true -20 1969-12-31 16:00:09.969 -20.0 821UdmGbkEf4j 20 -20 0 -20.0 -0.0 20.0 -1591.06 0.0 20.0 0.0 -10.175 -20.0 1.314 -20.0 0.0 -20 -NULL 34 1969-12-31 16:00:04.36 34.0 NULL -34 34 0 34.0 0.0 -34.0 2704.8019999999997 0.0 -34.0 0.0 -10.175 34.0 -0.7729411764705882 34.0 0.0 34 -NULL 3 1969-12-31 16:00:07.648 3.0 NULL -3 3 0 3.0 0.0 -3.0 238.659 0.0 -3.0 0.0 -10.175 3.0 -8.76 3.0 0.0 3 -true 0 1969-12-31 16:00:07.615 0.0 1cGVWH7n1QU 0 0 0 0.0 0.0 -0.0 0.0 0.0 -0.0 0.0 -10.175 0.0 NULL 0.0 NULL 0 -true 55 1969-12-31 16:00:13.67 55.0 1cGVWH7n1QU -55 55 0 55.0 0.0 -55.0 4375.415 0.0 -55.0 0.0 -10.175 55.0 -0.47781818181818186 55.0 0.0 55 -NULL 50 1969-12-31 16:00:08.584 50.0 NULL -50 50 0 50.0 0.0 -50.0 3977.6499999999996 0.0 -50.0 0.0 -10.175 50.0 -0.5256000000000001 50.0 0.0 50 -NULL -3 1969-12-31 16:00:00.554 -3.0 NULL 3 -3 0 -3.0 -0.0 3.0 -238.659 0.0 3.0 0.0 -10.175 -3.0 8.76 -3.0 0.0 -3 -true 38 1969-12-31 16:00:00.78 38.0 821UdmGbkEf4j -38 38 0 38.0 0.0 -38.0 3023.014 0.0 -38.0 0.0 -10.175 38.0 -0.6915789473684211 38.0 0.0 38 -true 43 1969-12-31 16:00:12.613 43.0 821UdmGbkEf4j -43 43 0 43.0 0.0 -43.0 3420.779 0.0 -43.0 0.0 -10.175 43.0 -0.6111627906976744 43.0 0.0 43 -true -15 1969-12-31 16:00:09.694 -15.0 1cGVWH7n1QU 15 -15 0 -15.0 -0.0 15.0 -1193.295 0.0 15.0 0.0 -10.175 -15.0 1.752 -15.0 0.0 -15 -NULL -35 1969-12-31 16:00:13.605 -35.0 NULL 35 -35 0 -35.0 -0.0 35.0 -2784.355 0.0 35.0 0.0 -10.175 -35.0 0.7508571428571429 -35.0 0.0 -35 -NULL -47 1969-12-31 16:00:04.351 -47.0 NULL 47 -47 0 -47.0 -0.0 47.0 -3738.991 0.0 47.0 0.0 -10.175 -47.0 0.5591489361702128 -47.0 0.0 -47 -NULL -10 1969-12-31 16:00:06.264 -10.0 NULL 10 -10 0 -10.0 -0.0 10.0 -795.53 0.0 10.0 0.0 -10.175 -10.0 2.628 -10.0 0.0 -10 -NULL -42 1969-12-31 16:00:08.553 -42.0 NULL 42 -42 0 -42.0 -0.0 42.0 -3341.2259999999997 0.0 42.0 0.0 -10.175 -42.0 0.6257142857142858 -42.0 0.0 -42 -true 10 1969-12-31 16:00:07.636 10.0 821UdmGbkEf4j -10 10 0 10.0 0.0 -10.0 795.53 0.0 -10.0 0.0 -10.175 10.0 -2.628 10.0 0.0 10 -NULL -54 1969-12-31 16:00:13.616 -54.0 NULL 54 -54 0 -54.0 -0.0 54.0 -4295.862 0.0 54.0 0.0 -10.175 -54.0 0.4866666666666667 -54.0 0.0 -54 -true -46 1969-12-31 16:00:04.272 -46.0 1cGVWH7n1QU 46 -46 0 -46.0 -0.0 46.0 -3659.438 0.0 46.0 0.0 -10.175 -46.0 0.571304347826087 -46.0 0.0 -46 -true 3 1969-12-31 16:00:04.64 3.0 1cGVWH7n1QU -3 3 0 3.0 0.0 -3.0 238.659 0.0 -3.0 0.0 -10.175 3.0 -8.76 3.0 0.0 3 -NULL -3 1969-12-31 16:00:07.754 -3.0 NULL 3 -3 0 -3.0 -0.0 3.0 -238.659 0.0 3.0 0.0 -10.175 -3.0 8.76 -3.0 0.0 -3 -NULL -26 1969-12-31 16:00:08.33 -26.0 NULL 26 -26 0 -26.0 -0.0 26.0 -2068.3779999999997 0.0 26.0 0.0 -10.175 -26.0 1.0107692307692309 -26.0 0.0 -26 -NULL -41 1969-12-31 16:00:05.785 -41.0 NULL 41 -41 0 -41.0 -0.0 41.0 -3261.673 0.0 41.0 0.0 -10.175 -41.0 0.6409756097560976 -41.0 0.0 -41 -NULL 20 1969-12-31 16:00:01.267 20.0 NULL -20 20 0 20.0 0.0 -20.0 1591.06 0.0 -20.0 0.0 -10.175 20.0 -1.314 20.0 0.0 20 -true 4 1969-12-31 16:00:12.526 4.0 821UdmGbkEf4j -4 4 0 4.0 0.0 -4.0 318.212 0.0 -4.0 0.0 -10.175 4.0 -6.57 4.0 0.0 4 -NULL -27 1969-12-31 16:00:11.72 -27.0 NULL 27 -27 0 -27.0 -0.0 27.0 -2147.931 0.0 27.0 0.0 -10.175 -27.0 0.9733333333333334 -27.0 0.0 -27 -NULL 32 1969-12-31 16:00:13.823 32.0 NULL -32 32 0 32.0 0.0 -32.0 2545.696 0.0 -32.0 0.0 -10.175 32.0 -0.82125 32.0 0.0 32 -NULL 52 1969-12-31 16:00:06.616 52.0 NULL -52 52 0 52.0 0.0 -52.0 4136.755999999999 0.0 -52.0 0.0 -10.175 52.0 -0.5053846153846154 52.0 0.0 52 -NULL 35 1969-12-31 16:00:00.812 35.0 NULL -35 35 0 35.0 0.0 -35.0 2784.355 0.0 -35.0 0.0 -10.175 35.0 -0.7508571428571429 35.0 0.0 35 -true -12 1969-12-31 16:00:14.907 -12.0 cvLH6Eat2yFsyy7p 12 -12 0 -12.0 -0.0 12.0 -954.636 0.0 12.0 0.0 -10.175 -12.0 2.19 -12.0 0.0 -12 -NULL -23 1969-12-31 16:00:03.695 -26.28 NULL 23 -23 0 -26.280000686645508 -0.0 26.280000686645508 -2090.65289462471 0.0 26.280000686645508 0.0 -10.175 -26.280000686645508 0.9999999738719373 -26.28 0.0 -23 -NULL 2 1969-12-31 16:00:12.06 2.0 NULL -2 2 0 2.0 0.0 -2.0 159.106 0.0 -2.0 0.0 -10.175 2.0 -13.14 2.0 0.0 2 -true 56 1969-12-31 16:00:08.683 56.0 821UdmGbkEf4j -56 56 0 56.0 0.0 -56.0 4454.968 0.0 -56.0 0.0 -10.175 56.0 -0.4692857142857143 56.0 0.0 56 -NULL -56 1969-12-31 16:00:09.818 -56.0 NULL 56 -56 0 -56.0 -0.0 56.0 -4454.968 0.0 56.0 0.0 -10.175 -56.0 0.4692857142857143 -56.0 0.0 -56 -NULL 42 1969-12-31 16:00:00.11 42.0 NULL -42 42 0 42.0 0.0 -42.0 3341.2259999999997 0.0 -42.0 0.0 -10.175 42.0 -0.6257142857142858 42.0 0.0 42 -true 7 1969-12-31 16:00:01.437 7.0 1cGVWH7n1QU -7 7 0 7.0 0.0 -7.0 556.871 0.0 -7.0 0.0 -10.175 7.0 -3.7542857142857144 7.0 0.0 7 -NULL 29 1969-12-31 16:00:08.001 29.0 NULL -29 29 0 29.0 0.0 -29.0 2307.037 0.0 -29.0 0.0 -10.175 29.0 -0.9062068965517242 29.0 0.0 29 -NULL 51 1969-12-31 16:00:01.211 51.0 NULL -51 51 0 51.0 0.0 -51.0 4057.203 0.0 -51.0 0.0 -10.175 51.0 -0.5152941176470588 51.0 0.0 51 -true -49 1969-12-31 16:00:05.159 -49.0 821UdmGbkEf4j 49 -49 0 -49.0 -0.0 49.0 -3898.0969999999998 0.0 49.0 0.0 -10.175 -49.0 0.5363265306122449 -49.0 0.0 -49 -true 43 1969-12-31 16:00:06.596 43.0 cvLH6Eat2yFsyy7p -43 43 0 43.0 0.0 -43.0 3420.779 0.0 -43.0 0.0 -10.175 43.0 -0.6111627906976744 43.0 0.0 43 -NULL 18 1969-12-31 16:00:07.846 18.0 NULL -18 18 0 18.0 0.0 -18.0 1431.954 0.0 -18.0 0.0 -10.175 18.0 -1.46 18.0 0.0 18 -true -11 1969-12-31 16:00:14.985 -11.0 cvLH6Eat2yFsyy7p 11 -11 0 -11.0 -0.0 11.0 -875.083 0.0 11.0 0.0 -10.175 -11.0 2.389090909090909 -11.0 0.0 -11 -true -9 1969-12-31 16:00:06.798 -9.0 cvLH6Eat2yFsyy7p 9 -9 0 -9.0 -0.0 9.0 -715.977 0.0 9.0 0.0 -10.175 -9.0 2.92 -9.0 0.0 -9 -NULL 9 1969-12-31 16:00:31.808 9.0 NULL -9 9 0 9.0 0.0 -9.0 715.977 0.0 -9.0 0.0 -10.175 9.0 -2.92 9.0 0.0 9 -true 54 1969-12-31 16:00:04.268 54.0 821UdmGbkEf4j -54 54 0 54.0 0.0 -54.0 4295.862 0.0 -54.0 0.0 -10.175 54.0 -0.4866666666666667 54.0 0.0 54 -NULL 48 1969-12-31 16:00:10.669 48.0 NULL -48 48 0 48.0 0.0 -48.0 3818.544 0.0 -48.0 0.0 -10.175 48.0 -0.5475 48.0 0.0 48 -true -32 1969-12-31 16:00:02.585 -32.0 cvLH6Eat2yFsyy7p 32 -32 0 -32.0 -0.0 32.0 -2545.696 0.0 32.0 0.0 -10.175 -32.0 0.82125 -32.0 0.0 -32 -NULL 31 1969-12-31 16:00:03.219 31.0 NULL -31 31 0 31.0 0.0 -31.0 2466.143 0.0 -31.0 0.0 -10.175 31.0 -0.847741935483871 31.0 0.0 31 -true 23 1969-12-31 16:00:14.752 23.0 1cGVWH7n1QU -23 23 0 23.0 0.0 -23.0 1829.719 0.0 -23.0 0.0 -10.175 23.0 -1.142608695652174 23.0 0.0 23 -NULL -44 1969-12-31 16:00:00.835 -44.0 NULL 44 -44 0 -44.0 -0.0 44.0 -3500.332 0.0 44.0 0.0 -10.175 -44.0 0.5972727272727273 -44.0 0.0 -44 -true 41 1969-12-31 16:00:00.805 41.0 1cGVWH7n1QU -41 41 0 41.0 0.0 -41.0 3261.673 0.0 -41.0 0.0 -10.175 41.0 -0.6409756097560976 41.0 0.0 41 -NULL 30 1969-12-31 16:00:03.875 30.0 NULL -30 30 0 30.0 0.0 -30.0 2386.59 0.0 -30.0 0.0 -10.175 30.0 -0.876 30.0 0.0 30 -NULL 4 1969-12-31 16:00:04.207 4.0 NULL -4 4 0 4.0 0.0 -4.0 318.212 0.0 -4.0 0.0 -10.175 4.0 -6.57 4.0 0.0 4 -NULL 8 1969-12-31 16:00:06.469 8.0 NULL -8 8 0 8.0 0.0 -8.0 636.424 0.0 -8.0 0.0 -10.175 8.0 -3.285 8.0 0.0 8 -NULL 39 1969-12-31 16:00:12.719 39.0 NULL -39 39 0 39.0 0.0 -39.0 3102.567 0.0 -39.0 0.0 -10.175 39.0 -0.6738461538461539 39.0 0.0 39 -true -40 1969-12-31 16:00:14.282 -40.0 821UdmGbkEf4j 40 -40 0 -40.0 -0.0 40.0 -3182.12 0.0 40.0 0.0 -10.175 -40.0 0.657 -40.0 0.0 -40 -NULL 42 1969-12-31 16:00:05.173 42.0 NULL -42 42 0 42.0 0.0 -42.0 3341.2259999999997 0.0 -42.0 0.0 -10.175 42.0 -0.6257142857142858 42.0 0.0 42 -NULL 57 1969-12-31 16:00:07.361 57.0 NULL -57 57 0 57.0 0.0 -57.0 4534.521 0.0 -57.0 0.0 -10.175 57.0 -0.4610526315789474 57.0 0.0 57 -true 46 1969-12-31 16:00:16.276 46.0 821UdmGbkEf4j -46 46 0 46.0 0.0 -46.0 3659.438 0.0 -46.0 0.0 -10.175 46.0 -0.571304347826087 46.0 0.0 46 -NULL -8 1969-12-31 16:00:08.783 -8.0 NULL 8 -8 0 -8.0 -0.0 8.0 -636.424 0.0 8.0 0.0 -10.175 -8.0 3.285 -8.0 0.0 -8 -true 43 1969-12-31 16:00:02.844 43.0 cvLH6Eat2yFsyy7p -43 43 0 43.0 0.0 -43.0 3420.779 0.0 -43.0 0.0 -10.175 43.0 -0.6111627906976744 43.0 0.0 43 -true 56 1969-12-31 16:00:05 56.0 821UdmGbkEf4j -56 56 0 56.0 0.0 -56.0 4454.968 0.0 -56.0 0.0 -10.175 56.0 -0.4692857142857143 56.0 0.0 56 -true -50 1969-12-31 16:00:00.274 -50.0 821UdmGbkEf4j 50 -50 0 -50.0 -0.0 50.0 -3977.6499999999996 0.0 50.0 0.0 -10.175 -50.0 0.5256000000000001 -50.0 0.0 -50 -true 22 1969-12-31 16:00:12.849 22.0 821UdmGbkEf4j -22 22 0 22.0 0.0 -22.0 1750.166 0.0 -22.0 0.0 -10.175 22.0 -1.1945454545454546 22.0 0.0 22 -NULL 24 1969-12-31 16:00:01.369 24.0 NULL -24 24 0 24.0 0.0 -24.0 1909.272 0.0 -24.0 0.0 -10.175 24.0 -1.095 24.0 0.0 24 -NULL 50 1969-12-31 16:00:05.051 50.0 NULL -50 50 0 50.0 0.0 -50.0 3977.6499999999996 0.0 -50.0 0.0 -10.175 50.0 -0.5256000000000001 50.0 0.0 50 -NULL -55 1969-12-31 16:00:11.751 -55.0 NULL 55 -55 0 -55.0 -0.0 55.0 -4375.415 0.0 55.0 0.0 -10.175 -55.0 0.47781818181818186 -55.0 0.0 -55 -NULL -32 1969-12-31 16:00:05.178 -32.0 NULL 32 -32 0 -32.0 -0.0 32.0 -2545.696 0.0 32.0 0.0 -10.175 -32.0 0.82125 -32.0 0.0 -32 -NULL 33 1969-12-31 16:00:01.27 33.0 NULL -33 33 0 33.0 0.0 -33.0 2625.249 0.0 -33.0 0.0 -10.175 33.0 -0.7963636363636364 33.0 0.0 33 -NULL 52 1969-12-31 16:00:15.227 52.0 NULL -52 52 0 52.0 0.0 -52.0 4136.755999999999 0.0 -52.0 0.0 -10.175 52.0 -0.5053846153846154 52.0 0.0 52 -NULL 32 1969-12-31 16:00:08.961 32.0 NULL -32 32 0 32.0 0.0 -32.0 2545.696 0.0 -32.0 0.0 -10.175 32.0 -0.82125 32.0 0.0 32 -NULL -39 1969-12-31 16:00:14.626 -39.0 NULL 39 -39 0 -39.0 -0.0 39.0 -3102.567 0.0 39.0 0.0 -10.175 -39.0 0.6738461538461539 -39.0 0.0 -39 -NULL -1 1969-12-31 16:00:04.994 -1.0 NULL 1 -1 0 -1.0 -0.0 1.0 -79.553 0.0 1.0 0.0 -10.175 -1.0 26.28 -1.0 0.0 -1 -NULL -4 1969-12-31 16:00:14.592 -4.0 NULL 4 -4 0 -4.0 -0.0 4.0 -318.212 0.0 4.0 0.0 -10.175 -4.0 6.57 -4.0 0.0 -4 -true 12 1969-12-31 16:00:00.997 12.0 821UdmGbkEf4j -12 12 0 12.0 0.0 -12.0 954.636 0.0 -12.0 0.0 -10.175 12.0 -2.19 12.0 0.0 12 -NULL 29 1969-12-31 16:00:14.829 29.0 NULL -29 29 0 29.0 0.0 -29.0 2307.037 0.0 -29.0 0.0 -10.175 29.0 -0.9062068965517242 29.0 0.0 29 -true 10 1969-12-31 16:00:06.923 10.0 821UdmGbkEf4j -10 10 0 10.0 0.0 -10.0 795.53 0.0 -10.0 0.0 -10.175 10.0 -2.628 10.0 0.0 10 -NULL -53 1969-12-31 16:00:08.176 -53.0 NULL 53 -53 0 -53.0 -0.0 53.0 -4216.309 0.0 53.0 0.0 -10.175 -53.0 0.4958490566037736 -53.0 0.0 -53 -true 25 1969-12-31 16:00:12.226 25.0 1cGVWH7n1QU -25 25 0 25.0 0.0 -25.0 1988.8249999999998 0.0 -25.0 0.0 -10.175 25.0 -1.0512000000000001 25.0 0.0 25 -NULL -26 1969-12-31 16:00:01.121 -26.0 NULL 26 -26 0 -26.0 -0.0 26.0 -2068.3779999999997 0.0 26.0 0.0 -10.175 -26.0 1.0107692307692309 -26.0 0.0 -26 -NULL -56 1969-12-31 16:00:03.273 -56.0 NULL 56 -56 0 -56.0 -0.0 56.0 -4454.968 0.0 56.0 0.0 -10.175 -56.0 0.4692857142857143 -56.0 0.0 -56 -NULL -20 1969-12-31 16:00:00.396 -20.0 NULL 20 -20 0 -20.0 -0.0 20.0 -1591.06 0.0 20.0 0.0 -10.175 -20.0 1.314 -20.0 0.0 -20 -true 22 1969-12-31 16:00:10.47 22.0 821UdmGbkEf4j -22 22 0 22.0 0.0 -22.0 1750.166 0.0 -22.0 0.0 -10.175 22.0 -1.1945454545454546 22.0 0.0 22 -true 58 1969-12-31 16:00:01.049 58.0 cvLH6Eat2yFsyy7p -58 58 0 58.0 0.0 -58.0 4614.074 0.0 -58.0 0.0 -10.175 58.0 -0.4531034482758621 58.0 0.0 58 -true 20 1969-12-31 16:00:14.818 20.0 1cGVWH7n1QU -20 20 0 20.0 0.0 -20.0 1591.06 0.0 -20.0 0.0 -10.175 20.0 -1.314 20.0 0.0 20 -NULL 5 1969-12-31 16:00:10.885 5.0 NULL -5 5 0 5.0 0.0 -5.0 397.765 0.0 -5.0 0.0 -10.175 5.0 -5.256 5.0 0.0 5 -true -57 1969-12-31 16:00:01.831 -57.0 1cGVWH7n1QU 57 -57 0 -57.0 -0.0 57.0 -4534.521 0.0 57.0 0.0 -10.175 -57.0 0.4610526315789474 -57.0 0.0 -57 -true 60 1969-12-31 16:00:01.679 60.0 1cGVWH7n1QU -60 60 0 60.0 0.0 -60.0 4773.18 0.0 -60.0 0.0 -10.175 60.0 -0.438 60.0 0.0 60 -NULL -58 1969-12-31 16:00:12.065 -58.0 NULL 58 -58 0 -58.0 -0.0 58.0 -4614.074 0.0 58.0 0.0 -10.175 -58.0 0.4531034482758621 -58.0 0.0 -58 -NULL -6 1969-12-31 16:00:05.495 -6.0 NULL 6 -6 0 -6.0 -0.0 6.0 -477.318 0.0 6.0 0.0 -10.175 -6.0 4.38 -6.0 0.0 -6 -true -49 1969-12-31 16:00:01.658 -49.0 cvLH6Eat2yFsyy7p 49 -49 0 -49.0 -0.0 49.0 -3898.0969999999998 0.0 49.0 0.0 -10.175 -49.0 0.5363265306122449 -49.0 0.0 -49 -NULL -38 1969-12-31 16:00:11.569 -38.0 NULL 38 -38 0 -38.0 -0.0 38.0 -3023.014 0.0 38.0 0.0 -10.175 -38.0 0.6915789473684211 -38.0 0.0 -38 -true -45 1969-12-31 16:00:11.903 -45.0 1cGVWH7n1QU 45 -45 0 -45.0 -0.0 45.0 -3579.8849999999998 0.0 45.0 0.0 -10.175 -45.0 0.5840000000000001 -45.0 0.0 -45 -NULL 21 1969-12-31 16:00:00.418 21.0 NULL -21 21 0 21.0 0.0 -21.0 1670.6129999999998 0.0 -21.0 0.0 -10.175 21.0 -1.2514285714285716 21.0 0.0 21 -true 16 1969-12-31 16:00:15.698 16.0 1cGVWH7n1QU -16 16 0 16.0 0.0 -16.0 1272.848 0.0 -16.0 0.0 -10.175 16.0 -1.6425 16.0 0.0 16 -NULL -53 1969-12-31 16:00:11.36 -53.0 NULL 53 -53 0 -53.0 -0.0 53.0 -4216.309 0.0 53.0 0.0 -10.175 -53.0 0.4958490566037736 -53.0 0.0 -53 -true -41 1969-12-31 16:00:03.19 -41.0 1cGVWH7n1QU 41 -41 0 -41.0 -0.0 41.0 -3261.673 0.0 41.0 0.0 -10.175 -41.0 0.6409756097560976 -41.0 0.0 -41 -NULL 28 1969-12-31 16:00:06.092 28.0 NULL -28 28 0 28.0 0.0 -28.0 2227.484 0.0 -28.0 0.0 -10.175 28.0 -0.9385714285714286 28.0 0.0 28 -NULL 5 1969-12-31 16:00:06.877 5.0 NULL -5 5 0 5.0 0.0 -5.0 397.765 0.0 -5.0 0.0 -10.175 5.0 -5.256 5.0 0.0 5 -NULL 1 1969-12-31 16:00:15.883 1.0 NULL -1 1 0 1.0 0.0 -1.0 79.553 0.0 -1.0 0.0 -10.175 1.0 -26.28 1.0 0.0 1 -true 49 1969-12-31 16:00:13.889 49.0 cvLH6Eat2yFsyy7p -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 -true -32 1969-12-31 16:00:15.275 -32.0 821UdmGbkEf4j 32 -32 0 -32.0 -0.0 32.0 -2545.696 0.0 32.0 0.0 -10.175 -32.0 0.82125 -32.0 0.0 -32 -true -31 1969-12-31 16:00:10.653 -31.0 821UdmGbkEf4j 31 -31 0 -31.0 -0.0 31.0 -2466.143 0.0 31.0 0.0 -10.175 -31.0 0.847741935483871 -31.0 0.0 -31 -NULL -62 1969-12-31 16:00:10.485 -62.0 NULL 62 -62 0 -62.0 -0.0 62.0 -4932.286 0.0 62.0 0.0 -10.175 -62.0 0.4238709677419355 -62.0 0.0 -62 -NULL -8 1969-12-31 16:00:09.994 -8.0 NULL 8 -8 0 -8.0 -0.0 8.0 -636.424 0.0 8.0 0.0 -10.175 -8.0 3.285 -8.0 0.0 -8 -NULL -28 1969-12-31 16:00:02.58 -28.0 NULL 28 -28 0 -28.0 -0.0 28.0 -2227.484 0.0 28.0 0.0 -10.175 -28.0 0.9385714285714286 -28.0 0.0 -28 -true -27 1969-12-31 16:00:04.162 -27.0 cvLH6Eat2yFsyy7p 27 -27 0 -27.0 -0.0 27.0 -2147.931 0.0 27.0 0.0 -10.175 -27.0 0.9733333333333334 -27.0 0.0 -27 -true -35 1969-12-31 16:00:15.328 -35.0 821UdmGbkEf4j 35 -35 0 -35.0 -0.0 35.0 -2784.355 0.0 35.0 0.0 -10.175 -35.0 0.7508571428571429 -35.0 0.0 -35 -true -60 1969-12-31 16:00:11.849 -60.0 1cGVWH7n1QU 60 -60 0 -60.0 -0.0 60.0 -4773.18 0.0 60.0 0.0 -10.175 -60.0 0.438 -60.0 0.0 -60 -true -40 1969-12-31 16:00:11.833 -40.0 cvLH6Eat2yFsyy7p 40 -40 0 -40.0 -0.0 40.0 -3182.12 0.0 40.0 0.0 -10.175 -40.0 0.657 -40.0 0.0 -40 -NULL 12 1969-12-31 16:00:09.556 12.0 NULL -12 12 0 12.0 0.0 -12.0 954.636 0.0 -12.0 0.0 -10.175 12.0 -2.19 12.0 0.0 12 -NULL -60 1969-12-31 16:00:10.953 -60.0 NULL 60 -60 0 -60.0 -0.0 60.0 -4773.18 0.0 60.0 0.0 -10.175 -60.0 0.438 -60.0 0.0 -60 -NULL 60 1969-12-31 16:00:13.214 60.0 NULL -60 60 0 60.0 0.0 -60.0 4773.18 0.0 -60.0 0.0 -10.175 60.0 -0.438 60.0 0.0 60 -NULL 6 1969-12-31 16:00:02.826 6.0 NULL -6 6 0 6.0 0.0 -6.0 477.318 0.0 -6.0 0.0 -10.175 6.0 -4.38 6.0 0.0 6 -NULL -19 1969-12-31 16:00:10.814 -19.0 NULL 19 -19 0 -19.0 -0.0 19.0 -1511.507 0.0 19.0 0.0 -10.175 -19.0 1.3831578947368421 -19.0 0.0 -19 -true -21 1969-12-31 16:00:10.437 -21.0 1cGVWH7n1QU 21 -21 0 -21.0 -0.0 21.0 -1670.6129999999998 0.0 21.0 0.0 -10.175 -21.0 1.2514285714285716 -21.0 0.0 -21 -true 24 1969-12-31 16:00:04.096 24.0 cvLH6Eat2yFsyy7p -24 24 0 24.0 0.0 -24.0 1909.272 0.0 -24.0 0.0 -10.175 24.0 -1.095 24.0 0.0 24 -true -44 1969-12-31 16:00:15.658 -44.0 1cGVWH7n1QU 44 -44 0 -44.0 -0.0 44.0 -3500.332 0.0 44.0 0.0 -10.175 -44.0 0.5972727272727273 -44.0 0.0 -44 -true -60 1969-12-31 16:00:13.567 -60.0 821UdmGbkEf4j 60 -60 0 -60.0 -0.0 60.0 -4773.18 0.0 60.0 0.0 -10.175 -60.0 0.438 -60.0 0.0 -60 -NULL 22 1969-12-31 16:00:07.249 22.0 NULL -22 22 0 22.0 0.0 -22.0 1750.166 0.0 -22.0 0.0 -10.175 22.0 -1.1945454545454546 22.0 0.0 22 -NULL -25 1969-12-31 16:00:03.328 -25.0 NULL 25 -25 0 -25.0 -0.0 25.0 -1988.8249999999998 0.0 25.0 0.0 -10.175 -25.0 1.0512000000000001 -25.0 0.0 -25 -NULL 57 1969-12-31 16:00:07.907 57.0 NULL -57 57 0 57.0 0.0 -57.0 4534.521 0.0 -57.0 0.0 -10.175 57.0 -0.4610526315789474 57.0 0.0 57 -NULL -62 1969-12-31 16:00:00.037 -62.0 NULL 62 -62 0 -62.0 -0.0 62.0 -4932.286 0.0 62.0 0.0 -10.175 -62.0 0.4238709677419355 -62.0 0.0 -62 -true 49 1969-12-31 16:00:07.648 49.0 cvLH6Eat2yFsyy7p -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 -NULL 8 1969-12-31 16:00:00.664 8.0 NULL -8 8 0 8.0 0.0 -8.0 636.424 0.0 -8.0 0.0 -10.175 8.0 -3.285 8.0 0.0 8 -NULL -28 1969-12-31 16:00:13.922 -28.0 NULL 28 -28 0 -28.0 -0.0 28.0 -2227.484 0.0 28.0 0.0 -10.175 -28.0 0.9385714285714286 -28.0 0.0 -28 -NULL -4 1969-12-31 16:00:16.558 -4.0 NULL 4 -4 0 -4.0 -0.0 4.0 -318.212 0.0 4.0 0.0 -10.175 -4.0 6.57 -4.0 0.0 -4 -NULL -56 1969-12-31 16:00:01.468 -56.0 NULL 56 -56 0 -56.0 -0.0 56.0 -4454.968 0.0 56.0 0.0 -10.175 -56.0 0.4692857142857143 -56.0 0.0 -56 -NULL 46 1969-12-31 16:00:05.226 46.0 NULL -46 46 0 46.0 0.0 -46.0 3659.438 0.0 -46.0 0.0 -10.175 46.0 -0.571304347826087 46.0 0.0 46 -true 2 1969-12-31 16:00:07.06 2.0 821UdmGbkEf4j -2 2 0 2.0 0.0 -2.0 159.106 0.0 -2.0 0.0 -10.175 2.0 -13.14 2.0 0.0 2 -NULL 21 1969-12-31 16:00:15.417 21.0 NULL -21 21 0 21.0 0.0 -21.0 1670.6129999999998 0.0 -21.0 0.0 -10.175 21.0 -1.2514285714285716 21.0 0.0 21 -NULL -19 1969-12-31 16:00:01.781 -19.0 NULL 19 -19 0 -19.0 -0.0 19.0 -1511.507 0.0 19.0 0.0 -10.175 -19.0 1.3831578947368421 -19.0 0.0 -19 -true 2 1969-12-31 16:00:16.352 2.0 821UdmGbkEf4j -2 2 0 2.0 0.0 -2.0 159.106 0.0 -2.0 0.0 -10.175 2.0 -13.14 2.0 0.0 2 -NULL -15 1969-12-31 16:00:09.027 -15.0 NULL 15 -15 0 -15.0 -0.0 15.0 -1193.295 0.0 15.0 0.0 -10.175 -15.0 1.752 -15.0 0.0 -15 -NULL -51 1969-12-31 16:00:13.197 -51.0 NULL 51 -51 0 -51.0 -0.0 51.0 -4057.203 0.0 51.0 0.0 -10.175 -51.0 0.5152941176470588 -51.0 0.0 -51 -NULL 47 1969-12-31 16:00:00.351 47.0 NULL -47 47 0 47.0 0.0 -47.0 3738.991 0.0 -47.0 0.0 -10.175 47.0 -0.5591489361702128 47.0 0.0 47 -NULL -29 1969-12-31 16:00:11.762 -29.0 NULL 29 -29 0 -29.0 -0.0 29.0 -2307.037 0.0 29.0 0.0 -10.175 -29.0 0.9062068965517242 -29.0 0.0 -29 -NULL 25 1969-12-31 16:00:03.599 25.0 NULL -25 25 0 25.0 0.0 -25.0 1988.8249999999998 0.0 -25.0 0.0 -10.175 25.0 -1.0512000000000001 25.0 0.0 25 -true -37 1969-12-31 16:00:05.855 -37.0 cvLH6Eat2yFsyy7p 37 -37 0 -37.0 -0.0 37.0 -2943.461 0.0 37.0 0.0 -10.175 -37.0 0.7102702702702703 -37.0 0.0 -37 -NULL -27 1969-12-31 16:00:08.832 -27.0 NULL 27 -27 0 -27.0 -0.0 27.0 -2147.931 0.0 27.0 0.0 -10.175 -27.0 0.9733333333333334 -27.0 0.0 -27 -NULL 31 1969-12-31 16:00:05.892 31.0 NULL -31 31 0 31.0 0.0 -31.0 2466.143 0.0 -31.0 0.0 -10.175 31.0 -0.847741935483871 31.0 0.0 31 -NULL 15 1969-12-31 16:00:11.15 15.0 NULL -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -NULL -8 1969-12-31 16:00:03.136 -8.0 NULL 8 -8 0 -8.0 -0.0 8.0 -636.424 0.0 8.0 0.0 -10.175 -8.0 3.285 -8.0 0.0 -8 -true 15 1969-12-31 16:00:12.059 15.0 821UdmGbkEf4j -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -NULL 26 1969-12-31 16:00:00.547 26.0 NULL -26 26 0 26.0 0.0 -26.0 2068.3779999999997 0.0 -26.0 0.0 -10.175 26.0 -1.0107692307692309 26.0 0.0 26 -true 17 1969-12-31 16:00:01.764 17.0 821UdmGbkEf4j -17 17 0 17.0 0.0 -17.0 1352.4009999999998 0.0 -17.0 0.0 -10.175 17.0 -1.5458823529411765 17.0 0.0 17 -true -61 1969-12-31 16:00:02.339 -61.0 cvLH6Eat2yFsyy7p 61 -61 0 -61.0 -0.0 61.0 -4852.733 0.0 61.0 0.0 -10.175 -61.0 0.4308196721311476 -61.0 0.0 -61 -true 22 1969-12-31 16:00:04.322 22.0 821UdmGbkEf4j -22 22 0 22.0 0.0 -22.0 1750.166 0.0 -22.0 0.0 -10.175 22.0 -1.1945454545454546 22.0 0.0 22 -NULL 49 1969-12-31 16:00:04.12 49.0 NULL -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 -NULL 10 1969-12-31 16:00:13.961 10.0 NULL -10 10 0 10.0 0.0 -10.0 795.53 0.0 -10.0 0.0 -10.175 10.0 -2.628 10.0 0.0 10 -true 19 1969-12-31 16:00:11.452 19.0 1cGVWH7n1QU -19 19 0 19.0 0.0 -19.0 1511.507 0.0 -19.0 0.0 -10.175 19.0 -1.3831578947368421 19.0 0.0 19 -NULL 61 1969-12-31 16:00:07.588 61.0 NULL -61 61 0 61.0 0.0 -61.0 4852.733 0.0 -61.0 0.0 -10.175 61.0 -0.4308196721311476 61.0 0.0 61 -NULL -48 1969-12-31 16:00:03.188 -48.0 NULL 48 -48 0 -48.0 -0.0 48.0 -3818.544 0.0 48.0 0.0 -10.175 -48.0 0.5475 -48.0 0.0 -48 -true 15 1969-12-31 16:00:02.598 15.0 1cGVWH7n1QU -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -NULL -57 1969-12-31 16:00:02.592 -57.0 NULL 57 -57 0 -57.0 -0.0 57.0 -4534.521 0.0 57.0 0.0 -10.175 -57.0 0.4610526315789474 -57.0 0.0 -57 -true 48 1969-12-31 16:00:01.512 48.0 821UdmGbkEf4j -48 48 0 48.0 0.0 -48.0 3818.544 0.0 -48.0 0.0 -10.175 48.0 -0.5475 48.0 0.0 48 -true 49 1969-12-31 16:00:08.827 49.0 cvLH6Eat2yFsyy7p -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 -true 33 1969-12-31 16:00:04.672 33.0 1cGVWH7n1QU -33 33 0 33.0 0.0 -33.0 2625.249 0.0 -33.0 0.0 -10.175 33.0 -0.7963636363636364 33.0 0.0 33 -true -18 1969-12-31 16:00:08.837 -18.0 1cGVWH7n1QU 18 -18 0 -18.0 -0.0 18.0 -1431.954 0.0 18.0 0.0 -10.175 -18.0 1.46 -18.0 0.0 -18 -NULL 12 1969-12-31 16:00:08.34 12.0 NULL -12 12 0 12.0 0.0 -12.0 954.636 0.0 -12.0 0.0 -10.175 12.0 -2.19 12.0 0.0 12 -true 23 1969-12-31 16:00:01.779 23.0 1cGVWH7n1QU -23 23 0 23.0 0.0 -23.0 1829.719 0.0 -23.0 0.0 -10.175 23.0 -1.142608695652174 23.0 0.0 23 -true 15 1969-12-31 16:00:16.133 15.0 1cGVWH7n1QU -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -NULL -29 1969-12-31 16:00:03.116 -29.0 NULL 29 -29 0 -29.0 -0.0 29.0 -2307.037 0.0 29.0 0.0 -10.175 -29.0 0.9062068965517242 -29.0 0.0 -29 -true -21 1969-12-31 16:00:00.704 -21.0 821UdmGbkEf4j 21 -21 0 -21.0 -0.0 21.0 -1670.6129999999998 0.0 21.0 0.0 -10.175 -21.0 1.2514285714285716 -21.0 0.0 -21 -NULL 2 1969-12-31 16:00:01.489 2.0 NULL -2 2 0 2.0 0.0 -2.0 159.106 0.0 -2.0 0.0 -10.175 2.0 -13.14 2.0 0.0 2 -true 48 1969-12-31 16:00:15.654 48.0 1cGVWH7n1QU -48 48 0 48.0 0.0 -48.0 3818.544 0.0 -48.0 0.0 -10.175 48.0 -0.5475 48.0 0.0 48 -NULL -32 1969-12-31 16:00:11.928 -32.0 NULL 32 -32 0 -32.0 -0.0 32.0 -2545.696 0.0 32.0 0.0 -10.175 -32.0 0.82125 -32.0 0.0 -32 -NULL 40 1969-12-31 16:00:00.258 40.0 NULL -40 40 0 40.0 0.0 -40.0 3182.12 0.0 -40.0 0.0 -10.175 40.0 -0.657 40.0 0.0 40 -NULL 28 1969-12-31 16:00:11.858 28.0 NULL -28 28 0 28.0 0.0 -28.0 2227.484 0.0 -28.0 0.0 -10.175 28.0 -0.9385714285714286 28.0 0.0 28 -true 34 1969-12-31 16:00:09.488 34.0 1cGVWH7n1QU -34 34 0 34.0 0.0 -34.0 2704.8019999999997 0.0 -34.0 0.0 -10.175 34.0 -0.7729411764705882 34.0 0.0 34 -true -62 1969-12-31 16:00:07.847 -62.0 cvLH6Eat2yFsyy7p 62 -62 0 -62.0 -0.0 62.0 -4932.286 0.0 62.0 0.0 -10.175 -62.0 0.4238709677419355 -62.0 0.0 -62 -true -64 1969-12-31 16:00:00.172 -64.0 1cGVWH7n1QU 64 -64 0 -64.0 -0.0 64.0 -5091.392 0.0 64.0 0.0 -10.175 -64.0 0.410625 -64.0 0.0 -64 -true 49 1969-12-31 16:00:05.028 49.0 821UdmGbkEf4j -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 -NULL 30 1969-12-31 16:00:05.066 30.0 NULL -30 30 0 30.0 0.0 -30.0 2386.59 0.0 -30.0 0.0 -10.175 30.0 -0.876 30.0 0.0 30 -NULL 36 1969-12-31 16:00:06.51 36.0 NULL -36 36 0 36.0 0.0 -36.0 2863.908 0.0 -36.0 0.0 -10.175 36.0 -0.73 36.0 0.0 36 -NULL -54 1969-12-31 16:00:06.484 -54.0 NULL 54 -54 0 -54.0 -0.0 54.0 -4295.862 0.0 54.0 0.0 -10.175 -54.0 0.4866666666666667 -54.0 0.0 -54 -true 52 1969-12-31 16:00:00.199 52.0 821UdmGbkEf4j -52 52 0 52.0 0.0 -52.0 4136.755999999999 0.0 -52.0 0.0 -10.175 52.0 -0.5053846153846154 52.0 0.0 52 -NULL 45 1969-12-31 16:00:00.543 45.0 NULL -45 45 0 45.0 0.0 -45.0 3579.8849999999998 0.0 -45.0 0.0 -10.175 45.0 -0.5840000000000001 45.0 0.0 45 -NULL -64 1969-12-31 16:00:13.274 -64.0 NULL 64 -64 0 -64.0 -0.0 64.0 -5091.392 0.0 64.0 0.0 -10.175 -64.0 0.410625 -64.0 0.0 -64 -true -44 1969-12-31 16:00:10.921 -44.0 cvLH6Eat2yFsyy7p 44 -44 0 -44.0 -0.0 44.0 -3500.332 0.0 44.0 0.0 -10.175 -44.0 0.5972727272727273 -44.0 0.0 -44 -NULL -43 1969-12-31 16:00:00.026 -43.0 NULL 43 -43 0 -43.0 -0.0 43.0 -3420.779 0.0 43.0 0.0 -10.175 -43.0 0.6111627906976744 -43.0 0.0 -43 -NULL 33 1969-12-31 16:00:04.396 33.0 NULL -33 33 0 33.0 0.0 -33.0 2625.249 0.0 -33.0 0.0 -10.175 33.0 -0.7963636363636364 33.0 0.0 33 -true -15 1969-12-31 16:00:08.841 -15.0 cvLH6Eat2yFsyy7p 15 -15 0 -15.0 -0.0 15.0 -1193.295 0.0 15.0 0.0 -10.175 -15.0 1.752 -15.0 0.0 -15 -true -38 1969-12-31 16:00:11.654 -38.0 821UdmGbkEf4j 38 -38 0 -38.0 -0.0 38.0 -3023.014 0.0 38.0 0.0 -10.175 -38.0 0.6915789473684211 -38.0 0.0 -38 -true 23 1969-12-31 16:00:10.47 23.0 1cGVWH7n1QU -23 23 0 23.0 0.0 -23.0 1829.719 0.0 -23.0 0.0 -10.175 23.0 -1.142608695652174 23.0 0.0 23 -NULL -24 1969-12-31 16:00:14.558 -24.0 NULL 24 -24 0 -24.0 -0.0 24.0 -1909.272 0.0 24.0 0.0 -10.175 -24.0 1.095 -24.0 0.0 -24 -true 43 1969-12-31 16:00:01.028 43.0 821UdmGbkEf4j -43 43 0 43.0 0.0 -43.0 3420.779 0.0 -43.0 0.0 -10.175 43.0 -0.6111627906976744 43.0 0.0 43 -NULL 11 1969-12-31 16:00:16.044 11.0 NULL -11 11 0 11.0 0.0 -11.0 875.083 0.0 -11.0 0.0 -10.175 11.0 -2.389090909090909 11.0 0.0 11 -NULL 35 1969-12-31 16:00:01.792 35.0 NULL -35 35 0 35.0 0.0 -35.0 2784.355 0.0 -35.0 0.0 -10.175 35.0 -0.7508571428571429 35.0 0.0 35 -NULL 6 1969-12-31 16:00:15.958 6.0 NULL -6 6 0 6.0 0.0 -6.0 477.318 0.0 -6.0 0.0 -10.175 6.0 -4.38 6.0 0.0 6 -NULL -36 1969-12-31 16:00:06.072 -36.0 NULL 36 -36 0 -36.0 -0.0 36.0 -2863.908 0.0 36.0 0.0 -10.175 -36.0 0.73 -36.0 0.0 -36 -true 30 1969-12-31 16:00:07.457 30.0 821UdmGbkEf4j -30 30 0 30.0 0.0 -30.0 2386.59 0.0 -30.0 0.0 -10.175 30.0 -0.876 30.0 0.0 30 -NULL 46 1969-12-31 16:00:04.717 46.0 NULL -46 46 0 46.0 0.0 -46.0 3659.438 0.0 -46.0 0.0 -10.175 46.0 -0.571304347826087 46.0 0.0 46 -NULL 32 1969-12-31 16:00:15.868 32.0 NULL -32 32 0 32.0 0.0 -32.0 2545.696 0.0 -32.0 0.0 -10.175 32.0 -0.82125 32.0 0.0 32 -true 56 1969-12-31 16:00:13.259 56.0 cvLH6Eat2yFsyy7p -56 56 0 56.0 0.0 -56.0 4454.968 0.0 -56.0 0.0 -10.175 56.0 -0.4692857142857143 56.0 0.0 56 -NULL 58 1969-12-31 16:00:14.013 58.0 NULL -58 58 0 58.0 0.0 -58.0 4614.074 0.0 -58.0 0.0 -10.175 58.0 -0.4531034482758621 58.0 0.0 58 -true -7 1969-12-31 16:00:04.368 -7.0 cvLH6Eat2yFsyy7p 7 -7 0 -7.0 -0.0 7.0 -556.871 0.0 7.0 0.0 -10.175 -7.0 3.7542857142857144 -7.0 0.0 -7 -true -26 1969-12-31 16:00:16.28 -26.0 821UdmGbkEf4j 26 -26 0 -26.0 -0.0 26.0 -2068.3779999999997 0.0 26.0 0.0 -10.175 -26.0 1.0107692307692309 -26.0 0.0 -26 -true 23 1969-12-31 16:00:10.907 23.0 821UdmGbkEf4j -23 23 0 23.0 0.0 -23.0 1829.719 0.0 -23.0 0.0 -10.175 23.0 -1.142608695652174 23.0 0.0 23 -true 38 1969-12-31 16:00:09.366 38.0 cvLH6Eat2yFsyy7p -38 38 0 38.0 0.0 -38.0 3023.014 0.0 -38.0 0.0 -10.175 38.0 -0.6915789473684211 38.0 0.0 38 -true 38 1969-12-31 16:00:06.103 38.0 cvLH6Eat2yFsyy7p -38 38 0 38.0 0.0 -38.0 3023.014 0.0 -38.0 0.0 -10.175 38.0 -0.6915789473684211 38.0 0.0 38 -NULL -56 1969-12-31 16:00:02.298 -56.0 NULL 56 -56 0 -56.0 -0.0 56.0 -4454.968 0.0 56.0 0.0 -10.175 -56.0 0.4692857142857143 -56.0 0.0 -56 -true 11 1969-12-31 16:00:07.581 11.0 1cGVWH7n1QU -11 11 0 11.0 0.0 -11.0 875.083 0.0 -11.0 0.0 -10.175 11.0 -2.389090909090909 11.0 0.0 11 -NULL 6 1969-12-31 16:00:13.132 6.0 NULL -6 6 0 6.0 0.0 -6.0 477.318 0.0 -6.0 0.0 -10.175 6.0 -4.38 6.0 0.0 6 -true -24 1969-12-31 16:00:06.349 -24.0 821UdmGbkEf4j 24 -24 0 -24.0 -0.0 24.0 -1909.272 0.0 24.0 0.0 -10.175 -24.0 1.095 -24.0 0.0 -24 -NULL 35 1969-12-31 16:00:07.613 35.0 NULL -35 35 0 35.0 0.0 -35.0 2784.355 0.0 -35.0 0.0 -10.175 35.0 -0.7508571428571429 35.0 0.0 35 -true 2 1969-12-31 16:00:07.062 2.0 cvLH6Eat2yFsyy7p -2 2 0 2.0 0.0 -2.0 159.106 0.0 -2.0 0.0 -10.175 2.0 -13.14 2.0 0.0 2 -NULL -57 1969-12-31 16:00:09.1 -57.0 NULL 57 -57 0 -57.0 -0.0 57.0 -4534.521 0.0 57.0 0.0 -10.175 -57.0 0.4610526315789474 -57.0 0.0 -57 -true 9 1969-12-31 16:00:14.974 9.0 cvLH6Eat2yFsyy7p -9 9 0 9.0 0.0 -9.0 715.977 0.0 -9.0 0.0 -10.175 9.0 -2.92 9.0 0.0 9 -true 7 1969-12-31 16:00:16.238 7.0 cvLH6Eat2yFsyy7p -7 7 0 7.0 0.0 -7.0 556.871 0.0 -7.0 0.0 -10.175 7.0 -3.7542857142857144 7.0 0.0 7 -true 57 1969-12-31 16:00:07.705 57.0 cvLH6Eat2yFsyy7p -57 57 0 57.0 0.0 -57.0 4534.521 0.0 -57.0 0.0 -10.175 57.0 -0.4610526315789474 57.0 0.0 57 -NULL 41 1969-12-31 16:00:01.951 41.0 NULL -41 41 0 41.0 0.0 -41.0 3261.673 0.0 -41.0 0.0 -10.175 41.0 -0.6409756097560976 41.0 0.0 41 -NULL 53 1969-12-31 16:00:13.231 53.0 NULL -53 53 0 53.0 0.0 -53.0 4216.309 0.0 -53.0 0.0 -10.175 53.0 -0.4958490566037736 53.0 0.0 53 -true -18 1969-12-31 16:00:04.617 -18.0 821UdmGbkEf4j 18 -18 0 -18.0 -0.0 18.0 -1431.954 0.0 18.0 0.0 -10.175 -18.0 1.46 -18.0 0.0 -18 -NULL 16 1969-12-31 16:00:08.966 16.0 NULL -16 16 0 16.0 0.0 -16.0 1272.848 0.0 -16.0 0.0 -10.175 16.0 -1.6425 16.0 0.0 16 -NULL 46 1969-12-31 16:00:09.441 46.0 NULL -46 46 0 46.0 0.0 -46.0 3659.438 0.0 -46.0 0.0 -10.175 46.0 -0.571304347826087 46.0 0.0 46 -true -48 1969-12-31 16:00:11.189 -48.0 821UdmGbkEf4j 48 -48 0 -48.0 -0.0 48.0 -3818.544 0.0 48.0 0.0 -10.175 -48.0 0.5475 -48.0 0.0 -48 -true 16 1969-12-31 16:00:00.98 16.0 1cGVWH7n1QU -16 16 0 16.0 0.0 -16.0 1272.848 0.0 -16.0 0.0 -10.175 16.0 -1.6425 16.0 0.0 16 -true -24 1969-12-31 16:00:07.494 -24.0 1cGVWH7n1QU 24 -24 0 -24.0 -0.0 24.0 -1909.272 0.0 24.0 0.0 -10.175 -24.0 1.095 -24.0 0.0 -24 -true 34 1969-12-31 16:00:10.23 34.0 821UdmGbkEf4j -34 34 0 34.0 0.0 -34.0 2704.8019999999997 0.0 -34.0 0.0 -10.175 34.0 -0.7729411764705882 34.0 0.0 34 -NULL 29 1969-12-31 16:00:04.204 29.0 NULL -29 29 0 29.0 0.0 -29.0 2307.037 0.0 -29.0 0.0 -10.175 29.0 -0.9062068965517242 29.0 0.0 29 -NULL 27 1969-12-31 16:00:13.951 27.0 NULL -27 27 0 27.0 0.0 -27.0 2147.931 0.0 -27.0 0.0 -10.175 27.0 -0.9733333333333334 27.0 0.0 27 -true 61 1969-12-31 16:00:08.3 61.0 1cGVWH7n1QU -61 61 0 61.0 0.0 -61.0 4852.733 0.0 -61.0 0.0 -10.175 61.0 -0.4308196721311476 61.0 0.0 61 -NULL 60 1969-12-31 16:00:07.707 60.0 NULL -60 60 0 60.0 0.0 -60.0 4773.18 0.0 -60.0 0.0 -10.175 60.0 -0.438 60.0 0.0 60 -NULL 37 1969-12-31 16:00:14.747 37.0 NULL -37 37 0 37.0 0.0 -37.0 2943.461 0.0 -37.0 0.0 -10.175 37.0 -0.7102702702702703 37.0 0.0 37 -true 15 1969-12-31 16:00:11.832 15.0 1cGVWH7n1QU -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -true -5 1969-12-31 16:00:00.594 -5.0 821UdmGbkEf4j 5 -5 0 -5.0 -0.0 5.0 -397.765 0.0 5.0 0.0 -10.175 -5.0 5.256 -5.0 0.0 -5 -NULL -18 1969-12-31 16:00:00.959 -18.0 NULL 18 -18 0 -18.0 -0.0 18.0 -1431.954 0.0 18.0 0.0 -10.175 -18.0 1.46 -18.0 0.0 -18 -NULL -21 1969-12-31 16:00:09.052 -21.0 NULL 21 -21 0 -21.0 -0.0 21.0 -1670.6129999999998 0.0 21.0 0.0 -10.175 -21.0 1.2514285714285716 -21.0 0.0 -21 -true -28 1969-12-31 16:00:14.761 -28.0 821UdmGbkEf4j 28 -28 0 -28.0 -0.0 28.0 -2227.484 0.0 28.0 0.0 -10.175 -28.0 0.9385714285714286 -28.0 0.0 -28 -true 53 1969-12-31 16:00:07.35 53.0 cvLH6Eat2yFsyy7p -53 53 0 53.0 0.0 -53.0 4216.309 0.0 -53.0 0.0 -10.175 53.0 -0.4958490566037736 53.0 0.0 53 -true 54 1969-12-31 16:00:16.558 54.0 1cGVWH7n1QU -54 54 0 54.0 0.0 -54.0 4295.862 0.0 -54.0 0.0 -10.175 54.0 -0.4866666666666667 54.0 0.0 54 -NULL 23 1969-12-31 16:00:00.103 23.0 NULL -23 23 0 23.0 0.0 -23.0 1829.719 0.0 -23.0 0.0 -10.175 23.0 -1.142608695652174 23.0 0.0 23 -NULL -12 1969-12-31 16:00:09.088 -12.0 NULL 12 -12 0 -12.0 -0.0 12.0 -954.636 0.0 12.0 0.0 -10.175 -12.0 2.19 -12.0 0.0 -12 -true -27 1969-12-31 16:00:01.346 -27.0 cvLH6Eat2yFsyy7p 27 -27 0 -27.0 -0.0 27.0 -2147.931 0.0 27.0 0.0 -10.175 -27.0 0.9733333333333334 -27.0 0.0 -27 -true -57 1969-12-31 16:00:14.225 -57.0 821UdmGbkEf4j 57 -57 0 -57.0 -0.0 57.0 -4534.521 0.0 57.0 0.0 -10.175 -57.0 0.4610526315789474 -57.0 0.0 -57 -true 6 1969-12-31 16:00:09.388 6.0 821UdmGbkEf4j -6 6 0 6.0 0.0 -6.0 477.318 0.0 -6.0 0.0 -10.175 6.0 -4.38 6.0 0.0 6 -NULL -57 1969-12-31 16:00:06.845 -57.0 NULL 57 -57 0 -57.0 -0.0 57.0 -4534.521 0.0 57.0 0.0 -10.175 -57.0 0.4610526315789474 -57.0 0.0 -57 -NULL -52 1969-12-31 16:00:13.014 -52.0 NULL 52 -52 0 -52.0 -0.0 52.0 -4136.755999999999 0.0 52.0 0.0 -10.175 -52.0 0.5053846153846154 -52.0 0.0 -52 -NULL -61 1969-12-31 16:00:11.842 -61.0 NULL 61 -61 0 -61.0 -0.0 61.0 -4852.733 0.0 61.0 0.0 -10.175 -61.0 0.4308196721311476 -61.0 0.0 -61 -NULL -11 1969-12-31 16:00:07.962 -11.0 NULL 11 -11 0 -11.0 -0.0 11.0 -875.083 0.0 11.0 0.0 -10.175 -11.0 2.389090909090909 -11.0 0.0 -11 -NULL 4 1969-12-31 16:00:07.759 4.0 NULL -4 4 0 4.0 0.0 -4.0 318.212 0.0 -4.0 0.0 -10.175 4.0 -6.57 4.0 0.0 4 -true 6 1969-12-31 16:00:07.375 6.0 1cGVWH7n1QU -6 6 0 6.0 0.0 -6.0 477.318 0.0 -6.0 0.0 -10.175 6.0 -4.38 6.0 0.0 6 -NULL -26 1969-12-31 16:00:14.315 -26.0 NULL 26 -26 0 -26.0 -0.0 26.0 -2068.3779999999997 0.0 26.0 0.0 -10.175 -26.0 1.0107692307692309 -26.0 0.0 -26 -true -46 1969-12-31 16:00:14.854 -46.0 cvLH6Eat2yFsyy7p 46 -46 0 -46.0 -0.0 46.0 -3659.438 0.0 46.0 0.0 -10.175 -46.0 0.571304347826087 -46.0 0.0 -46 -NULL 37 1969-12-31 16:00:11.309 37.0 NULL -37 37 0 37.0 0.0 -37.0 2943.461 0.0 -37.0 0.0 -10.175 37.0 -0.7102702702702703 37.0 0.0 37 -true -15 1969-12-31 16:00:16.004 -15.0 1cGVWH7n1QU 15 -15 0 -15.0 -0.0 15.0 -1193.295 0.0 15.0 0.0 -10.175 -15.0 1.752 -15.0 0.0 -15 -true -25 1969-12-31 16:00:13.381 -25.0 1cGVWH7n1QU 25 -25 0 -25.0 -0.0 25.0 -1988.8249999999998 0.0 25.0 0.0 -10.175 -25.0 1.0512000000000001 -25.0 0.0 -25 -NULL 21 1969-12-31 16:00:13.153 21.0 NULL -21 21 0 21.0 0.0 -21.0 1670.6129999999998 0.0 -21.0 0.0 -10.175 21.0 -1.2514285714285716 21.0 0.0 21 -true 46 1969-12-31 16:00:02.015 46.0 cvLH6Eat2yFsyy7p -46 46 0 46.0 0.0 -46.0 3659.438 0.0 -46.0 0.0 -10.175 46.0 -0.571304347826087 46.0 0.0 46 -NULL -28 1969-12-31 16:00:00.157 -28.0 NULL 28 -28 0 -28.0 -0.0 28.0 -2227.484 0.0 28.0 0.0 -10.175 -28.0 0.9385714285714286 -28.0 0.0 -28 -NULL -47 1969-12-31 16:00:10.649 -47.0 NULL 47 -47 0 -47.0 -0.0 47.0 -3738.991 0.0 47.0 0.0 -10.175 -47.0 0.5591489361702128 -47.0 0.0 -47 -true 50 1969-12-31 16:00:15.938 50.0 1cGVWH7n1QU -50 50 0 50.0 0.0 -50.0 3977.6499999999996 0.0 -50.0 0.0 -10.175 50.0 -0.5256000000000001 50.0 0.0 50 -NULL 31 1969-12-31 16:00:06.729 31.0 NULL -31 31 0 31.0 0.0 -31.0 2466.143 0.0 -31.0 0.0 -10.175 31.0 -0.847741935483871 31.0 0.0 31 -true 36 1969-12-31 16:00:04.432 36.0 cvLH6Eat2yFsyy7p -36 36 0 36.0 0.0 -36.0 2863.908 0.0 -36.0 0.0 -10.175 36.0 -0.73 36.0 0.0 36 -true 3 1969-12-31 16:00:14.349 3.0 cvLH6Eat2yFsyy7p -3 3 0 3.0 0.0 -3.0 238.659 0.0 -3.0 0.0 -10.175 3.0 -8.76 3.0 0.0 3 -NULL 22 1969-12-31 16:00:10.436 22.0 NULL -22 22 0 22.0 0.0 -22.0 1750.166 0.0 -22.0 0.0 -10.175 22.0 -1.1945454545454546 22.0 0.0 22 -NULL 11 1969-12-31 16:00:16.197 11.0 NULL -11 11 0 11.0 0.0 -11.0 875.083 0.0 -11.0 0.0 -10.175 11.0 -2.389090909090909 11.0 0.0 11 -true -40 1969-12-31 16:00:12.957 -40.0 1cGVWH7n1QU 40 -40 0 -40.0 -0.0 40.0 -3182.12 0.0 40.0 0.0 -10.175 -40.0 0.657 -40.0 0.0 -40 -true -46 1969-12-31 16:00:16.298 -46.0 cvLH6Eat2yFsyy7p 46 -46 0 -46.0 -0.0 46.0 -3659.438 0.0 46.0 0.0 -10.175 -46.0 0.571304347826087 -46.0 0.0 -46 -NULL -6 1969-12-31 16:00:00.052 -6.0 NULL 6 -6 0 -6.0 -0.0 6.0 -477.318 0.0 6.0 0.0 -10.175 -6.0 4.38 -6.0 0.0 -6 -NULL 36 1969-12-31 16:00:14.011 36.0 NULL -36 36 0 36.0 0.0 -36.0 2863.908 0.0 -36.0 0.0 -10.175 36.0 -0.73 36.0 0.0 36 -true -62 1969-12-31 16:00:02.922 -62.0 821UdmGbkEf4j 62 -62 0 -62.0 -0.0 62.0 -4932.286 0.0 62.0 0.0 -10.175 -62.0 0.4238709677419355 -62.0 0.0 -62 -NULL -54 1969-12-31 16:00:01.14 -54.0 NULL 54 -54 0 -54.0 -0.0 54.0 -4295.862 0.0 54.0 0.0 -10.175 -54.0 0.4866666666666667 -54.0 0.0 -54 -true -64 1969-12-31 16:00:01.79 -64.0 1cGVWH7n1QU 64 -64 0 -64.0 -0.0 64.0 -5091.392 0.0 64.0 0.0 -10.175 -64.0 0.410625 -64.0 0.0 -64 -true -38 1969-12-31 16:00:02.382 -38.0 1cGVWH7n1QU 38 -38 0 -38.0 -0.0 38.0 -3023.014 0.0 38.0 0.0 -10.175 -38.0 0.6915789473684211 -38.0 0.0 -38 -true -14 1969-12-31 16:00:07.207 -14.0 821UdmGbkEf4j 14 -14 0 -14.0 -0.0 14.0 -1113.742 0.0 14.0 0.0 -10.175 -14.0 1.8771428571428572 -14.0 0.0 -14 -NULL -30 1969-12-31 16:00:14.921 -30.0 NULL 30 -30 0 -30.0 -0.0 30.0 -2386.59 0.0 30.0 0.0 -10.175 -30.0 0.876 -30.0 0.0 -30 -NULL -4 1969-12-31 16:00:05.056 -4.0 NULL 4 -4 0 -4.0 -0.0 4.0 -318.212 0.0 4.0 0.0 -10.175 -4.0 6.57 -4.0 0.0 -4 -NULL 41 1969-12-31 16:00:08.408 41.0 NULL -41 41 0 41.0 0.0 -41.0 3261.673 0.0 -41.0 0.0 -10.175 41.0 -0.6409756097560976 41.0 0.0 41 -NULL 0 1969-12-31 16:00:13.039 0.0 NULL 0 0 0 0.0 0.0 -0.0 0.0 0.0 -0.0 0.0 -10.175 0.0 NULL 0.0 NULL 0 -NULL -11 1969-12-31 16:00:11.515 -11.0 NULL 11 -11 0 -11.0 -0.0 11.0 -875.083 0.0 11.0 0.0 -10.175 -11.0 2.389090909090909 -11.0 0.0 -11 -NULL -11 1969-12-31 16:00:13.638 -11.0 NULL 11 -11 0 -11.0 -0.0 11.0 -875.083 0.0 11.0 0.0 -10.175 -11.0 2.389090909090909 -11.0 0.0 -11 -true 20 1969-12-31 16:00:04.545 20.0 1cGVWH7n1QU -20 20 0 20.0 0.0 -20.0 1591.06 0.0 -20.0 0.0 -10.175 20.0 -1.314 20.0 0.0 20 -NULL -21 1969-12-31 16:00:14.073 -21.0 NULL 21 -21 0 -21.0 -0.0 21.0 -1670.6129999999998 0.0 21.0 0.0 -10.175 -21.0 1.2514285714285716 -21.0 0.0 -21 -NULL 0 1969-12-31 16:00:10.24 0.0 NULL 0 0 0 0.0 0.0 -0.0 0.0 0.0 -0.0 0.0 -10.175 0.0 NULL 0.0 NULL 0 -NULL -64 1969-12-31 16:00:03.944 -64.0 NULL 64 -64 0 -64.0 -0.0 64.0 -5091.392 0.0 64.0 0.0 -10.175 -64.0 0.410625 -64.0 0.0 -64 -NULL 33 1969-12-31 16:00:02.983 33.0 NULL -33 33 0 33.0 0.0 -33.0 2625.249 0.0 -33.0 0.0 -10.175 33.0 -0.7963636363636364 33.0 0.0 33 -NULL 43 1969-12-31 16:00:00.104 43.0 NULL -43 43 0 43.0 0.0 -43.0 3420.779 0.0 -43.0 0.0 -10.175 43.0 -0.6111627906976744 43.0 0.0 43 -true -32 1969-12-31 16:00:01.642 -32.0 cvLH6Eat2yFsyy7p 32 -32 0 -32.0 -0.0 32.0 -2545.696 0.0 32.0 0.0 -10.175 -32.0 0.82125 -32.0 0.0 -32 -NULL 45 1969-12-31 16:00:16.167 45.0 NULL -45 45 0 45.0 0.0 -45.0 3579.8849999999998 0.0 -45.0 0.0 -10.175 45.0 -0.5840000000000001 45.0 0.0 45 -NULL -58 1969-12-31 16:00:15.658 -58.0 NULL 58 -58 0 -58.0 -0.0 58.0 -4614.074 0.0 58.0 0.0 -10.175 -58.0 0.4531034482758621 -58.0 0.0 -58 -true 40 1969-12-31 16:00:02.643 40.0 821UdmGbkEf4j -40 40 0 40.0 0.0 -40.0 3182.12 0.0 -40.0 0.0 -10.175 40.0 -0.657 40.0 0.0 40 -true 28 1969-12-31 16:00:04.975 28.0 1cGVWH7n1QU -28 28 0 28.0 0.0 -28.0 2227.484 0.0 -28.0 0.0 -10.175 28.0 -0.9385714285714286 28.0 0.0 28 -NULL -12 1969-12-31 16:00:04.154 -12.0 NULL 12 -12 0 -12.0 -0.0 12.0 -954.636 0.0 12.0 0.0 -10.175 -12.0 2.19 -12.0 0.0 -12 -true 29 1969-12-31 16:00:07.007 29.0 cvLH6Eat2yFsyy7p -29 29 0 29.0 0.0 -29.0 2307.037 0.0 -29.0 0.0 -10.175 29.0 -0.9062068965517242 29.0 0.0 29 -true -23 1969-12-31 16:00:08.804 -23.0 cvLH6Eat2yFsyy7p 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -NULL 17 1969-12-31 16:00:10.672 17.0 NULL -17 17 0 17.0 0.0 -17.0 1352.4009999999998 0.0 -17.0 0.0 -10.175 17.0 -1.5458823529411765 17.0 0.0 17 -NULL 52 1969-12-31 16:00:13.14 52.0 NULL -52 52 0 52.0 0.0 -52.0 4136.755999999999 0.0 -52.0 0.0 -10.175 52.0 -0.5053846153846154 52.0 0.0 52 -NULL -9 1969-12-31 16:00:02.075 -9.0 NULL 9 -9 0 -9.0 -0.0 9.0 -715.977 0.0 9.0 0.0 -10.175 -9.0 2.92 -9.0 0.0 -9 -NULL 55 1969-12-31 16:00:03.384 55.0 NULL -55 55 0 55.0 0.0 -55.0 4375.415 0.0 -55.0 0.0 -10.175 55.0 -0.47781818181818186 55.0 0.0 55 -NULL -2 1969-12-31 16:00:13.839 -2.0 NULL 2 -2 0 -2.0 -0.0 2.0 -159.106 0.0 2.0 0.0 -10.175 -2.0 13.14 -2.0 0.0 -2 -true 49 1969-12-31 16:00:04.128 49.0 821UdmGbkEf4j -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 -true 27 1969-12-31 16:00:16.319 27.0 cvLH6Eat2yFsyy7p -27 27 0 27.0 0.0 -27.0 2147.931 0.0 -27.0 0.0 -10.175 27.0 -0.9733333333333334 27.0 0.0 27 -NULL -11 1969-12-31 16:00:02.894 -11.0 NULL 11 -11 0 -11.0 -0.0 11.0 -875.083 0.0 11.0 0.0 -10.175 -11.0 2.389090909090909 -11.0 0.0 -11 -true -56 1969-12-31 16:00:14.689 -56.0 cvLH6Eat2yFsyy7p 56 -56 0 -56.0 -0.0 56.0 -4454.968 0.0 56.0 0.0 -10.175 -56.0 0.4692857142857143 -56.0 0.0 -56 -NULL 19 1969-12-31 16:00:14.464 19.0 NULL -19 19 0 19.0 0.0 -19.0 1511.507 0.0 -19.0 0.0 -10.175 19.0 -1.3831578947368421 19.0 0.0 19 -true 46 1969-12-31 16:00:13.074 46.0 cvLH6Eat2yFsyy7p -46 46 0 46.0 0.0 -46.0 3659.438 0.0 -46.0 0.0 -10.175 46.0 -0.571304347826087 46.0 0.0 46 -NULL -35 1969-12-31 16:00:12.966 -35.0 NULL 35 -35 0 -35.0 -0.0 35.0 -2784.355 0.0 35.0 0.0 -10.175 -35.0 0.7508571428571429 -35.0 0.0 -35 -true 6 1969-12-31 16:00:04.983 6.0 cvLH6Eat2yFsyy7p -6 6 0 6.0 0.0 -6.0 477.318 0.0 -6.0 0.0 -10.175 6.0 -4.38 6.0 0.0 6 -NULL 32 1969-12-31 16:00:06.576 32.0 NULL -32 32 0 32.0 0.0 -32.0 2545.696 0.0 -32.0 0.0 -10.175 32.0 -0.82125 32.0 0.0 32 -NULL -24 1969-12-31 16:00:16.03 -24.0 NULL 24 -24 0 -24.0 -0.0 24.0 -1909.272 0.0 24.0 0.0 -10.175 -24.0 1.095 -24.0 0.0 -24 -true 55 1969-12-31 16:00:00.672 55.0 821UdmGbkEf4j -55 55 0 55.0 0.0 -55.0 4375.415 0.0 -55.0 0.0 -10.175 55.0 -0.47781818181818186 55.0 0.0 55 -true -23 1969-12-31 16:00:14.9 -23.0 cvLH6Eat2yFsyy7p 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -NULL 21 1969-12-31 16:00:14.318 21.0 NULL -21 21 0 21.0 0.0 -21.0 1670.6129999999998 0.0 -21.0 0.0 -10.175 21.0 -1.2514285714285716 21.0 0.0 21 -true -27 1969-12-31 16:00:14.723 -27.0 1cGVWH7n1QU 27 -27 0 -27.0 -0.0 27.0 -2147.931 0.0 27.0 0.0 -10.175 -27.0 0.9733333333333334 -27.0 0.0 -27 -true -34 1969-12-31 16:00:02.696 -34.0 cvLH6Eat2yFsyy7p 34 -34 0 -34.0 -0.0 34.0 -2704.8019999999997 0.0 34.0 0.0 -10.175 -34.0 0.7729411764705882 -34.0 0.0 -34 -true 22 1969-12-31 16:00:01.809 22.0 cvLH6Eat2yFsyy7p -22 22 0 22.0 0.0 -22.0 1750.166 0.0 -22.0 0.0 -10.175 22.0 -1.1945454545454546 22.0 0.0 22 -NULL -42 1969-12-31 16:00:09.72 -42.0 NULL 42 -42 0 -42.0 -0.0 42.0 -3341.2259999999997 0.0 42.0 0.0 -10.175 -42.0 0.6257142857142858 -42.0 0.0 -42 -NULL -56 1969-12-31 16:00:15.038 -56.0 NULL 56 -56 0 -56.0 -0.0 56.0 -4454.968 0.0 56.0 0.0 -10.175 -56.0 0.4692857142857143 -56.0 0.0 -56 -true 22 1969-12-31 16:00:10.098 22.0 1cGVWH7n1QU -22 22 0 22.0 0.0 -22.0 1750.166 0.0 -22.0 0.0 -10.175 22.0 -1.1945454545454546 22.0 0.0 22 -NULL 48 1969-12-31 16:00:02.043 48.0 NULL -48 48 0 48.0 0.0 -48.0 3818.544 0.0 -48.0 0.0 -10.175 48.0 -0.5475 48.0 0.0 48 -true 3 1969-12-31 16:00:03.192 3.0 821UdmGbkEf4j -3 3 0 3.0 0.0 -3.0 238.659 0.0 -3.0 0.0 -10.175 3.0 -8.76 3.0 0.0 3 -NULL -48 1969-12-31 16:00:06.337 -48.0 NULL 48 -48 0 -48.0 -0.0 48.0 -3818.544 0.0 48.0 0.0 -10.175 -48.0 0.5475 -48.0 0.0 -48 -true -45 1969-12-31 16:00:00.726 -45.0 821UdmGbkEf4j 45 -45 0 -45.0 -0.0 45.0 -3579.8849999999998 0.0 45.0 0.0 -10.175 -45.0 0.5840000000000001 -45.0 0.0 -45 -true -60 1969-12-31 16:00:06.389 -60.0 cvLH6Eat2yFsyy7p 60 -60 0 -60.0 -0.0 60.0 -4773.18 0.0 60.0 0.0 -10.175 -60.0 0.438 -60.0 0.0 -60 -true 45 1969-12-31 16:00:06.248 45.0 821UdmGbkEf4j -45 45 0 45.0 0.0 -45.0 3579.8849999999998 0.0 -45.0 0.0 -10.175 45.0 -0.5840000000000001 45.0 0.0 45 -NULL 26 1969-12-31 16:00:01.389 26.0 NULL -26 26 0 26.0 0.0 -26.0 2068.3779999999997 0.0 -26.0 0.0 -10.175 26.0 -1.0107692307692309 26.0 0.0 26 -NULL -40 1969-12-31 16:00:15.699 -40.0 NULL 40 -40 0 -40.0 -0.0 40.0 -3182.12 0.0 40.0 0.0 -10.175 -40.0 0.657 -40.0 0.0 -40 -NULL -44 1969-12-31 16:00:03.002 -44.0 NULL 44 -44 0 -44.0 -0.0 44.0 -3500.332 0.0 44.0 0.0 -10.175 -44.0 0.5972727272727273 -44.0 0.0 -44 -true 45 1969-12-31 16:00:13.953 45.0 cvLH6Eat2yFsyy7p -45 45 0 45.0 0.0 -45.0 3579.8849999999998 0.0 -45.0 0.0 -10.175 45.0 -0.5840000000000001 45.0 0.0 45 -NULL 37 1969-12-31 16:00:06.69 37.0 NULL -37 37 0 37.0 0.0 -37.0 2943.461 0.0 -37.0 0.0 -10.175 37.0 -0.7102702702702703 37.0 0.0 37 -true 38 1969-12-31 16:00:00.711 38.0 1cGVWH7n1QU -38 38 0 38.0 0.0 -38.0 3023.014 0.0 -38.0 0.0 -10.175 38.0 -0.6915789473684211 38.0 0.0 38 -true 24 1969-12-31 16:00:13.844 24.0 1cGVWH7n1QU -24 24 0 24.0 0.0 -24.0 1909.272 0.0 -24.0 0.0 -10.175 24.0 -1.095 24.0 0.0 24 -true 53 1969-12-31 16:00:14.694 53.0 cvLH6Eat2yFsyy7p -53 53 0 53.0 0.0 -53.0 4216.309 0.0 -53.0 0.0 -10.175 53.0 -0.4958490566037736 53.0 0.0 53 -NULL -21 1969-12-31 16:00:03.31 -21.0 NULL 21 -21 0 -21.0 -0.0 21.0 -1670.6129999999998 0.0 21.0 0.0 -10.175 -21.0 1.2514285714285716 -21.0 0.0 -21 -NULL 5 1969-12-31 16:00:10.973 5.0 NULL -5 5 0 5.0 0.0 -5.0 397.765 0.0 -5.0 0.0 -10.175 5.0 -5.256 5.0 0.0 5 -NULL -51 1969-12-31 16:00:02.723 -51.0 NULL 51 -51 0 -51.0 -0.0 51.0 -4057.203 0.0 51.0 0.0 -10.175 -51.0 0.5152941176470588 -51.0 0.0 -51 -NULL -63 1969-12-31 16:00:11.946 -63.0 NULL 63 -63 0 -63.0 -0.0 63.0 -5011.839 0.0 63.0 0.0 -10.175 -63.0 0.41714285714285715 -63.0 0.0 -63 -NULL -52 1969-12-31 16:00:05.377 -52.0 NULL 52 -52 0 -52.0 -0.0 52.0 -4136.755999999999 0.0 52.0 0.0 -10.175 -52.0 0.5053846153846154 -52.0 0.0 -52 -true -48 1969-12-31 16:00:02.921 -48.0 821UdmGbkEf4j 48 -48 0 -48.0 -0.0 48.0 -3818.544 0.0 48.0 0.0 -10.175 -48.0 0.5475 -48.0 0.0 -48 -true 45 1969-12-31 16:00:06.903 45.0 cvLH6Eat2yFsyy7p -45 45 0 45.0 0.0 -45.0 3579.8849999999998 0.0 -45.0 0.0 -10.175 45.0 -0.5840000000000001 45.0 0.0 45 -true 32 1969-12-31 16:00:12.525 32.0 cvLH6Eat2yFsyy7p -32 32 0 32.0 0.0 -32.0 2545.696 0.0 -32.0 0.0 -10.175 32.0 -0.82125 32.0 0.0 32 -NULL -5 1969-12-31 16:00:05.385 -5.0 NULL 5 -5 0 -5.0 -0.0 5.0 -397.765 0.0 5.0 0.0 -10.175 -5.0 5.256 -5.0 0.0 -5 -NULL 20 1969-12-31 16:00:00.54 20.0 NULL -20 20 0 20.0 0.0 -20.0 1591.06 0.0 -20.0 0.0 -10.175 20.0 -1.314 20.0 0.0 20 -true 7 1969-12-31 16:00:14.867 7.0 1cGVWH7n1QU -7 7 0 7.0 0.0 -7.0 556.871 0.0 -7.0 0.0 -10.175 7.0 -3.7542857142857144 7.0 0.0 7 -NULL -61 1969-12-31 16:00:16.558 -61.0 NULL 61 -61 0 -61.0 -0.0 61.0 -4852.733 0.0 61.0 0.0 -10.175 -61.0 0.4308196721311476 -61.0 0.0 -61 -true 17 1969-12-31 16:00:16.558 10.175 cvLH6Eat2yFsyy7p -17 17 0 10.175000190734863 0.0 -10.175000190734863 809.4517901735305 0.0 -10.175000190734863 0.0 -10.175 10.175000190734863 -2.58280093438524 10.175 0.0 17 -NULL -38 1969-12-31 16:00:00.843 -38.0 NULL 38 -38 0 -38.0 -0.0 38.0 -3023.014 0.0 38.0 0.0 -10.175 -38.0 0.6915789473684211 -38.0 0.0 -38 -NULL -25 1969-12-31 16:00:14.903 -25.0 NULL 25 -25 0 -25.0 -0.0 25.0 -1988.8249999999998 0.0 25.0 0.0 -10.175 -25.0 1.0512000000000001 -25.0 0.0 -25 -NULL -4 1969-12-31 16:00:13.865 -4.0 NULL 4 -4 0 -4.0 -0.0 4.0 -318.212 0.0 4.0 0.0 -10.175 -4.0 6.57 -4.0 0.0 -4 -true 14 1969-12-31 16:00:11.302 14.0 821UdmGbkEf4j -14 14 0 14.0 0.0 -14.0 1113.742 0.0 -14.0 0.0 -10.175 14.0 -1.8771428571428572 14.0 0.0 14 -NULL 38 1969-12-31 16:00:16.322 38.0 NULL -38 38 0 38.0 0.0 -38.0 3023.014 0.0 -38.0 0.0 -10.175 38.0 -0.6915789473684211 38.0 0.0 38 -NULL 59 1969-12-31 16:00:01.496 59.0 NULL -59 59 0 59.0 0.0 -59.0 4693.6269999999995 0.0 -59.0 0.0 -10.175 59.0 -0.44542372881355935 59.0 0.0 59 -NULL -42 1969-12-31 16:00:09.144 -42.0 NULL 42 -42 0 -42.0 -0.0 42.0 -3341.2259999999997 0.0 42.0 0.0 -10.175 -42.0 0.6257142857142858 -42.0 0.0 -42 -true 61 1969-12-31 16:00:12.397 61.0 1cGVWH7n1QU -61 61 0 61.0 0.0 -61.0 4852.733 0.0 -61.0 0.0 -10.175 61.0 -0.4308196721311476 61.0 0.0 61 -true -17 1969-12-31 16:00:09.292 -17.0 cvLH6Eat2yFsyy7p 17 -17 0 -17.0 -0.0 17.0 -1352.4009999999998 0.0 17.0 0.0 -10.175 -17.0 1.5458823529411765 -17.0 0.0 -17 -NULL -51 1969-12-31 16:00:07.536 -51.0 NULL 51 -51 0 -51.0 -0.0 51.0 -4057.203 0.0 51.0 0.0 -10.175 -51.0 0.5152941176470588 -51.0 0.0 -51 -true -7 1969-12-31 16:00:08.353 -7.0 cvLH6Eat2yFsyy7p 7 -7 0 -7.0 -0.0 7.0 -556.871 0.0 7.0 0.0 -10.175 -7.0 3.7542857142857144 -7.0 0.0 -7 -NULL 46 1969-12-31 16:00:06.353 46.0 NULL -46 46 0 46.0 0.0 -46.0 3659.438 0.0 -46.0 0.0 -10.175 46.0 -0.571304347826087 46.0 0.0 46 -true -26 1969-12-31 16:00:00.599 -26.0 cvLH6Eat2yFsyy7p 26 -26 0 -26.0 -0.0 26.0 -2068.3779999999997 0.0 26.0 0.0 -10.175 -26.0 1.0107692307692309 -26.0 0.0 -26 -true 19 1969-12-31 16:00:01.663 19.0 1cGVWH7n1QU -19 19 0 19.0 0.0 -19.0 1511.507 0.0 -19.0 0.0 -10.175 19.0 -1.3831578947368421 19.0 0.0 19 -true 62 1969-12-31 16:00:01.936 62.0 1cGVWH7n1QU -62 62 0 62.0 0.0 -62.0 4932.286 0.0 -62.0 0.0 -10.175 62.0 -0.4238709677419355 62.0 0.0 62 -true -56 1969-12-31 16:00:06.303 -56.0 821UdmGbkEf4j 56 -56 0 -56.0 -0.0 56.0 -4454.968 0.0 56.0 0.0 -10.175 -56.0 0.4692857142857143 -56.0 0.0 -56 -NULL -44 1969-12-31 16:00:15.522 -44.0 NULL 44 -44 0 -44.0 -0.0 44.0 -3500.332 0.0 44.0 0.0 -10.175 -44.0 0.5972727272727273 -44.0 0.0 -44 -NULL -2 1969-12-31 16:00:11.494 -2.0 NULL 2 -2 0 -2.0 -0.0 2.0 -159.106 0.0 2.0 0.0 -10.175 -2.0 13.14 -2.0 0.0 -2 -true 53 1969-12-31 16:00:00.51 53.0 821UdmGbkEf4j -53 53 0 53.0 0.0 -53.0 4216.309 0.0 -53.0 0.0 -10.175 53.0 -0.4958490566037736 53.0 0.0 53 -true 3 1969-12-31 16:00:10.295 3.0 821UdmGbkEf4j -3 3 0 3.0 0.0 -3.0 238.659 0.0 -3.0 0.0 -10.175 3.0 -8.76 3.0 0.0 3 -true -5 1969-12-31 16:00:06.123 -5.0 1cGVWH7n1QU 5 -5 0 -5.0 -0.0 5.0 -397.765 0.0 5.0 0.0 -10.175 -5.0 5.256 -5.0 0.0 -5 -true -5 1969-12-31 16:00:00.834 -5.0 cvLH6Eat2yFsyy7p 5 -5 0 -5.0 -0.0 5.0 -397.765 0.0 5.0 0.0 -10.175 -5.0 5.256 -5.0 0.0 -5 -NULL -59 1969-12-31 16:00:01.737 -59.0 NULL 59 -59 0 -59.0 -0.0 59.0 -4693.6269999999995 0.0 59.0 0.0 -10.175 -59.0 0.44542372881355935 -59.0 0.0 -59 -NULL -41 1969-12-31 16:00:08.011 -41.0 NULL 41 -41 0 -41.0 -0.0 41.0 -3261.673 0.0 41.0 0.0 -10.175 -41.0 0.6409756097560976 -41.0 0.0 -41 -true 9 1969-12-31 16:00:03.961 9.0 cvLH6Eat2yFsyy7p -9 9 0 9.0 0.0 -9.0 715.977 0.0 -9.0 0.0 -10.175 9.0 -2.92 9.0 0.0 9 -true 1 1969-12-31 16:00:00.342 1.0 1cGVWH7n1QU -1 1 0 1.0 0.0 -1.0 79.553 0.0 -1.0 0.0 -10.175 1.0 -26.28 1.0 0.0 1 -NULL 31 1969-12-31 16:00:02.119 31.0 NULL -31 31 0 31.0 0.0 -31.0 2466.143 0.0 -31.0 0.0 -10.175 31.0 -0.847741935483871 31.0 0.0 31 -NULL 24 1969-12-31 16:00:10.421 24.0 NULL -24 24 0 24.0 0.0 -24.0 1909.272 0.0 -24.0 0.0 -10.175 24.0 -1.095 24.0 0.0 24 -NULL -38 1969-12-31 16:00:02.007 -38.0 NULL 38 -38 0 -38.0 -0.0 38.0 -3023.014 0.0 38.0 0.0 -10.175 -38.0 0.6915789473684211 -38.0 0.0 -38 -true 2 1969-12-31 16:00:08.762 2.0 cvLH6Eat2yFsyy7p -2 2 0 2.0 0.0 -2.0 159.106 0.0 -2.0 0.0 -10.175 2.0 -13.14 2.0 0.0 2 -NULL -50 1969-12-31 16:00:13.663 -50.0 NULL 50 -50 0 -50.0 -0.0 50.0 -3977.6499999999996 0.0 50.0 0.0 -10.175 -50.0 0.5256000000000001 -50.0 0.0 -50 -NULL -44 1969-12-31 16:00:03.357 -44.0 NULL 44 -44 0 -44.0 -0.0 44.0 -3500.332 0.0 44.0 0.0 -10.175 -44.0 0.5972727272727273 -44.0 0.0 -44 -NULL 8 1969-12-31 16:00:11.315 8.0 NULL -8 8 0 8.0 0.0 -8.0 636.424 0.0 -8.0 0.0 -10.175 8.0 -3.285 8.0 0.0 8 -true 2 1969-12-31 16:00:03.805 2.0 821UdmGbkEf4j -2 2 0 2.0 0.0 -2.0 159.106 0.0 -2.0 0.0 -10.175 2.0 -13.14 2.0 0.0 2 -NULL 32 1969-12-31 16:00:07.155 32.0 NULL -32 32 0 32.0 0.0 -32.0 2545.696 0.0 -32.0 0.0 -10.175 32.0 -0.82125 32.0 0.0 32 -NULL 20 1969-12-31 16:00:05.722 20.0 NULL -20 20 0 20.0 0.0 -20.0 1591.06 0.0 -20.0 0.0 -10.175 20.0 -1.314 20.0 0.0 20 -true -53 1969-12-31 16:00:13.701 -53.0 cvLH6Eat2yFsyy7p 53 -53 0 -53.0 -0.0 53.0 -4216.309 0.0 53.0 0.0 -10.175 -53.0 0.4958490566037736 -53.0 0.0 -53 -true 4 1969-12-31 16:00:12.665 4.0 821UdmGbkEf4j -4 4 0 4.0 0.0 -4.0 318.212 0.0 -4.0 0.0 -10.175 4.0 -6.57 4.0 0.0 4 -true 20 1969-12-31 16:00:07.187 20.0 1cGVWH7n1QU -20 20 0 20.0 0.0 -20.0 1591.06 0.0 -20.0 0.0 -10.175 20.0 -1.314 20.0 0.0 20 -NULL -8 1969-12-31 16:00:08.007 -8.0 NULL 8 -8 0 -8.0 -0.0 8.0 -636.424 0.0 8.0 0.0 -10.175 -8.0 3.285 -8.0 0.0 -8 -NULL -30 1969-12-31 16:00:02.234 -30.0 NULL 30 -30 0 -30.0 -0.0 30.0 -2386.59 0.0 30.0 0.0 -10.175 -30.0 0.876 -30.0 0.0 -30 -true -52 1969-12-31 16:00:04.169 -52.0 821UdmGbkEf4j 52 -52 0 -52.0 -0.0 52.0 -4136.755999999999 0.0 52.0 0.0 -10.175 -52.0 0.5053846153846154 -52.0 0.0 -52 -NULL 42 1969-12-31 16:00:13.183 42.0 NULL -42 42 0 42.0 0.0 -42.0 3341.2259999999997 0.0 -42.0 0.0 -10.175 42.0 -0.6257142857142858 42.0 0.0 42 -true 38 1969-12-31 16:00:01.947 38.0 821UdmGbkEf4j -38 38 0 38.0 0.0 -38.0 3023.014 0.0 -38.0 0.0 -10.175 38.0 -0.6915789473684211 38.0 0.0 38 -NULL 55 1969-12-31 16:00:00.124 55.0 NULL -55 55 0 55.0 0.0 -55.0 4375.415 0.0 -55.0 0.0 -10.175 55.0 -0.47781818181818186 55.0 0.0 55 -NULL 33 1969-12-31 16:00:01.228 33.0 NULL -33 33 0 33.0 0.0 -33.0 2625.249 0.0 -33.0 0.0 -10.175 33.0 -0.7963636363636364 33.0 0.0 33 -true -31 1969-12-31 16:00:15.429 -31.0 cvLH6Eat2yFsyy7p 31 -31 0 -31.0 -0.0 31.0 -2466.143 0.0 31.0 0.0 -10.175 -31.0 0.847741935483871 -31.0 0.0 -31 -true 61 1969-12-31 16:00:07.974 61.0 cvLH6Eat2yFsyy7p -61 61 0 61.0 0.0 -61.0 4852.733 0.0 -61.0 0.0 -10.175 61.0 -0.4308196721311476 61.0 0.0 61 -true -55 1969-12-31 16:00:07.854 -55.0 1cGVWH7n1QU 55 -55 0 -55.0 -0.0 55.0 -4375.415 0.0 55.0 0.0 -10.175 -55.0 0.47781818181818186 -55.0 0.0 -55 -NULL -30 1969-12-31 16:00:00.434 -30.0 NULL 30 -30 0 -30.0 -0.0 30.0 -2386.59 0.0 30.0 0.0 -10.175 -30.0 0.876 -30.0 0.0 -30 -NULL 50 1969-12-31 16:00:09.965 50.0 NULL -50 50 0 50.0 0.0 -50.0 3977.6499999999996 0.0 -50.0 0.0 -10.175 50.0 -0.5256000000000001 50.0 0.0 50 -true 53 1969-12-31 16:00:00.751 53.0 1cGVWH7n1QU -53 53 0 53.0 0.0 -53.0 4216.309 0.0 -53.0 0.0 -10.175 53.0 -0.4958490566037736 53.0 0.0 53 -NULL 6 1969-12-31 16:00:07.204 6.0 NULL -6 6 0 6.0 0.0 -6.0 477.318 0.0 -6.0 0.0 -10.175 6.0 -4.38 6.0 0.0 6 -true -14 1969-12-31 16:00:09.002 -14.0 1cGVWH7n1QU 14 -14 0 -14.0 -0.0 14.0 -1113.742 0.0 14.0 0.0 -10.175 -14.0 1.8771428571428572 -14.0 0.0 -14 -NULL 45 1969-12-31 16:00:11.443 45.0 NULL -45 45 0 45.0 0.0 -45.0 3579.8849999999998 0.0 -45.0 0.0 -10.175 45.0 -0.5840000000000001 45.0 0.0 45 -NULL -49 1969-12-31 16:00:00.612 -49.0 NULL 49 -49 0 -49.0 -0.0 49.0 -3898.0969999999998 0.0 49.0 0.0 -10.175 -49.0 0.5363265306122449 -49.0 0.0 -49 -NULL 49 1969-12-31 16:00:03.074 49.0 NULL -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 -NULL -7 1969-12-31 16:00:04.342 -7.0 NULL 7 -7 0 -7.0 -0.0 7.0 -556.871 0.0 7.0 0.0 -10.175 -7.0 3.7542857142857144 -7.0 0.0 -7 -true -10 1969-12-31 16:00:00.986 -10.0 821UdmGbkEf4j 10 -10 0 -10.0 -0.0 10.0 -795.53 0.0 10.0 0.0 -10.175 -10.0 2.628 -10.0 0.0 -10 -NULL -1 1969-12-31 16:00:09.511 -1.0 NULL 1 -1 0 -1.0 -0.0 1.0 -79.553 0.0 1.0 0.0 -10.175 -1.0 26.28 -1.0 0.0 -1 -true -35 1969-12-31 16:00:05.647 -35.0 1cGVWH7n1QU 35 -35 0 -35.0 -0.0 35.0 -2784.355 0.0 35.0 0.0 -10.175 -35.0 0.7508571428571429 -35.0 0.0 -35 -NULL -17 1969-12-31 16:00:02.974 -17.0 NULL 17 -17 0 -17.0 -0.0 17.0 -1352.4009999999998 0.0 17.0 0.0 -10.175 -17.0 1.5458823529411765 -17.0 0.0 -17 -NULL 10 1969-12-31 16:00:09.22 10.0 NULL -10 10 0 10.0 0.0 -10.0 795.53 0.0 -10.0 0.0 -10.175 10.0 -2.628 10.0 0.0 10 -true 4 1969-12-31 16:00:08.036 4.0 1cGVWH7n1QU -4 4 0 4.0 0.0 -4.0 318.212 0.0 -4.0 0.0 -10.175 4.0 -6.57 4.0 0.0 4 -NULL -57 1969-12-31 16:00:00.833 -57.0 NULL 57 -57 0 -57.0 -0.0 57.0 -4534.521 0.0 57.0 0.0 -10.175 -57.0 0.4610526315789474 -57.0 0.0 -57 -NULL -2 1969-12-31 16:00:10.681 -2.0 NULL 2 -2 0 -2.0 -0.0 2.0 -159.106 0.0 2.0 0.0 -10.175 -2.0 13.14 -2.0 0.0 -2 -NULL 25 1969-12-31 16:00:12.473 25.0 NULL -25 25 0 25.0 0.0 -25.0 1988.8249999999998 0.0 -25.0 0.0 -10.175 25.0 -1.0512000000000001 25.0 0.0 25 -NULL 26 1969-12-31 16:00:08.152 26.0 NULL -26 26 0 26.0 0.0 -26.0 2068.3779999999997 0.0 -26.0 0.0 -10.175 26.0 -1.0107692307692309 26.0 0.0 26 -true 28 1969-12-31 16:00:15.571 28.0 1cGVWH7n1QU -28 28 0 28.0 0.0 -28.0 2227.484 0.0 -28.0 0.0 -10.175 28.0 -0.9385714285714286 28.0 0.0 28 -true 59 1969-12-31 16:00:04.322 59.0 cvLH6Eat2yFsyy7p -59 59 0 59.0 0.0 -59.0 4693.6269999999995 0.0 -59.0 0.0 -10.175 59.0 -0.44542372881355935 59.0 0.0 59 -NULL 8 1969-12-31 16:00:08.948 8.0 NULL -8 8 0 8.0 0.0 -8.0 636.424 0.0 -8.0 0.0 -10.175 8.0 -3.285 8.0 0.0 8 -true 60 1969-12-31 16:00:02.827 60.0 1cGVWH7n1QU -60 60 0 60.0 0.0 -60.0 4773.18 0.0 -60.0 0.0 -10.175 60.0 -0.438 60.0 0.0 60 -NULL 25 1969-12-31 16:00:02.722 25.0 NULL -25 25 0 25.0 0.0 -25.0 1988.8249999999998 0.0 -25.0 0.0 -10.175 25.0 -1.0512000000000001 25.0 0.0 25 -NULL 9 1969-12-31 16:00:12.205 9.0 NULL -9 9 0 9.0 0.0 -9.0 715.977 0.0 -9.0 0.0 -10.175 9.0 -2.92 9.0 0.0 9 -NULL 60 1969-12-31 16:00:07.419 60.0 NULL -60 60 0 60.0 0.0 -60.0 4773.18 0.0 -60.0 0.0 -10.175 60.0 -0.438 60.0 0.0 60 -NULL -36 1969-12-31 16:00:15.111 -36.0 NULL 36 -36 0 -36.0 -0.0 36.0 -2863.908 0.0 36.0 0.0 -10.175 -36.0 0.73 -36.0 0.0 -36 -true 8 1969-12-31 16:00:05.214 8.0 cvLH6Eat2yFsyy7p -8 8 0 8.0 0.0 -8.0 636.424 0.0 -8.0 0.0 -10.175 8.0 -3.285 8.0 0.0 8 -NULL -43 1969-12-31 16:00:03.341 -43.0 NULL 43 -43 0 -43.0 -0.0 43.0 -3420.779 0.0 43.0 0.0 -10.175 -43.0 0.6111627906976744 -43.0 0.0 -43 -NULL 21 1969-12-31 16:00:04.469 21.0 NULL -21 21 0 21.0 0.0 -21.0 1670.6129999999998 0.0 -21.0 0.0 -10.175 21.0 -1.2514285714285716 21.0 0.0 21 -true -7 1969-12-31 16:00:13.994 -7.0 cvLH6Eat2yFsyy7p 7 -7 0 -7.0 -0.0 7.0 -556.871 0.0 7.0 0.0 -10.175 -7.0 3.7542857142857144 -7.0 0.0 -7 -NULL 28 1969-12-31 16:00:02.795 28.0 NULL -28 28 0 28.0 0.0 -28.0 2227.484 0.0 -28.0 0.0 -10.175 28.0 -0.9385714285714286 28.0 0.0 28 -true -10 1969-12-31 16:00:09.51 -10.0 cvLH6Eat2yFsyy7p 10 -10 0 -10.0 -0.0 10.0 -795.53 0.0 10.0 0.0 -10.175 -10.0 2.628 -10.0 0.0 -10 -true 25 1969-12-31 16:00:00.925 25.0 1cGVWH7n1QU -25 25 0 25.0 0.0 -25.0 1988.8249999999998 0.0 -25.0 0.0 -10.175 25.0 -1.0512000000000001 25.0 0.0 25 -true 27 1969-12-31 16:00:12.725 27.0 1cGVWH7n1QU -27 27 0 27.0 0.0 -27.0 2147.931 0.0 -27.0 0.0 -10.175 27.0 -0.9733333333333334 27.0 0.0 27 -NULL 0 1969-12-31 16:00:10.139 0.0 NULL 0 0 0 0.0 0.0 -0.0 0.0 0.0 -0.0 0.0 -10.175 0.0 NULL 0.0 NULL 0 -true -55 1969-12-31 16:00:07.022 -55.0 821UdmGbkEf4j 55 -55 0 -55.0 -0.0 55.0 -4375.415 0.0 55.0 0.0 -10.175 -55.0 0.47781818181818186 -55.0 0.0 -55 -true -20 1969-12-31 16:00:09.451 -20.0 1cGVWH7n1QU 20 -20 0 -20.0 -0.0 20.0 -1591.06 0.0 20.0 0.0 -10.175 -20.0 1.314 -20.0 0.0 -20 -NULL 17 1969-12-31 16:00:05.562 17.0 NULL -17 17 0 17.0 0.0 -17.0 1352.4009999999998 0.0 -17.0 0.0 -10.175 17.0 -1.5458823529411765 17.0 0.0 17 -NULL 0 1969-12-31 16:00:05.959 0.0 NULL 0 0 0 0.0 0.0 -0.0 0.0 0.0 -0.0 0.0 -10.175 0.0 NULL 0.0 NULL 0 -NULL 32 1969-12-31 16:00:15.193 32.0 NULL -32 32 0 32.0 0.0 -32.0 2545.696 0.0 -32.0 0.0 -10.175 32.0 -0.82125 32.0 0.0 32 -NULL -49 1969-12-31 16:00:04.112 -49.0 NULL 49 -49 0 -49.0 -0.0 49.0 -3898.0969999999998 0.0 49.0 0.0 -10.175 -49.0 0.5363265306122449 -49.0 0.0 -49 -NULL 38 1969-12-31 16:00:05.29 38.0 NULL -38 38 0 38.0 0.0 -38.0 3023.014 0.0 -38.0 0.0 -10.175 38.0 -0.6915789473684211 38.0 0.0 38 -true -47 1969-12-31 16:00:06.297 -47.0 821UdmGbkEf4j 47 -47 0 -47.0 -0.0 47.0 -3738.991 0.0 47.0 0.0 -10.175 -47.0 0.5591489361702128 -47.0 0.0 -47 -true -36 1969-12-31 16:00:14.806 -36.0 821UdmGbkEf4j 36 -36 0 -36.0 -0.0 36.0 -2863.908 0.0 36.0 0.0 -10.175 -36.0 0.73 -36.0 0.0 -36 -true 42 1969-12-31 16:00:13.078 42.0 821UdmGbkEf4j -42 42 0 42.0 0.0 -42.0 3341.2259999999997 0.0 -42.0 0.0 -10.175 42.0 -0.6257142857142858 42.0 0.0 42 -true 23 1969-12-31 16:00:12 23.0 1cGVWH7n1QU -23 23 0 23.0 0.0 -23.0 1829.719 0.0 -23.0 0.0 -10.175 23.0 -1.142608695652174 23.0 0.0 23 -NULL 49 1969-12-31 16:00:12.977 49.0 NULL -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 -NULL 4 1969-12-31 16:00:08.327 4.0 NULL -4 4 0 4.0 0.0 -4.0 318.212 0.0 -4.0 0.0 -10.175 4.0 -6.57 4.0 0.0 4 -true -18 1969-12-31 16:00:11.231 -18.0 1cGVWH7n1QU 18 -18 0 -18.0 -0.0 18.0 -1431.954 0.0 18.0 0.0 -10.175 -18.0 1.46 -18.0 0.0 -18 -true -59 1969-12-31 16:00:11.109 -59.0 1cGVWH7n1QU 59 -59 0 -59.0 -0.0 59.0 -4693.6269999999995 0.0 59.0 0.0 -10.175 -59.0 0.44542372881355935 -59.0 0.0 -59 -true -52 1969-12-31 16:00:00.849 -52.0 1cGVWH7n1QU 52 -52 0 -52.0 -0.0 52.0 -4136.755999999999 0.0 52.0 0.0 -10.175 -52.0 0.5053846153846154 -52.0 0.0 -52 -true -48 1969-12-31 16:00:00.077 -48.0 cvLH6Eat2yFsyy7p 48 -48 0 -48.0 -0.0 48.0 -3818.544 0.0 48.0 0.0 -10.175 -48.0 0.5475 -48.0 0.0 -48 -NULL -31 1969-12-31 16:00:00.931 -31.0 NULL 31 -31 0 -31.0 -0.0 31.0 -2466.143 0.0 31.0 0.0 -10.175 -31.0 0.847741935483871 -31.0 0.0 -31 -true 32 1969-12-31 16:00:15.375 32.0 1cGVWH7n1QU -32 32 0 32.0 0.0 -32.0 2545.696 0.0 -32.0 0.0 -10.175 32.0 -0.82125 32.0 0.0 32 -true -25 1969-12-31 16:00:05.169 -25.0 821UdmGbkEf4j 25 -25 0 -25.0 -0.0 25.0 -1988.8249999999998 0.0 25.0 0.0 -10.175 -25.0 1.0512000000000001 -25.0 0.0 -25 -NULL 56 1969-12-31 16:00:09.452 56.0 NULL -56 56 0 56.0 0.0 -56.0 4454.968 0.0 -56.0 0.0 -10.175 56.0 -0.4692857142857143 56.0 0.0 56 -NULL 59 1969-12-31 16:00:05.13 59.0 NULL -59 59 0 59.0 0.0 -59.0 4693.6269999999995 0.0 -59.0 0.0 -10.175 59.0 -0.44542372881355935 59.0 0.0 59 -NULL 19 1969-12-31 16:00:02.483 19.0 NULL -19 19 0 19.0 0.0 -19.0 1511.507 0.0 -19.0 0.0 -10.175 19.0 -1.3831578947368421 19.0 0.0 19 -NULL -22 1969-12-31 16:00:13.315 -22.0 NULL 22 -22 0 -22.0 -0.0 22.0 -1750.166 0.0 22.0 0.0 -10.175 -22.0 1.1945454545454546 -22.0 0.0 -22 -true 28 1969-12-31 16:00:15.605 28.0 cvLH6Eat2yFsyy7p -28 28 0 28.0 0.0 -28.0 2227.484 0.0 -28.0 0.0 -10.175 28.0 -0.9385714285714286 28.0 0.0 28 -true -53 1969-12-31 16:00:11.598 -53.0 821UdmGbkEf4j 53 -53 0 -53.0 -0.0 53.0 -4216.309 0.0 53.0 0.0 -10.175 -53.0 0.4958490566037736 -53.0 0.0 -53 -NULL 2 1969-12-31 16:00:00.45 2.0 NULL -2 2 0 2.0 0.0 -2.0 159.106 0.0 -2.0 0.0 -10.175 2.0 -13.14 2.0 0.0 2 -true -1 1969-12-31 16:00:11.459 -1.0 821UdmGbkEf4j 1 -1 0 -1.0 -0.0 1.0 -79.553 0.0 1.0 0.0 -10.175 -1.0 26.28 -1.0 0.0 -1 -true -24 1969-12-31 16:00:06.636 -24.0 1cGVWH7n1QU 24 -24 0 -24.0 -0.0 24.0 -1909.272 0.0 24.0 0.0 -10.175 -24.0 1.095 -24.0 0.0 -24 -true -10 1969-12-31 16:00:02.372 -10.0 821UdmGbkEf4j 10 -10 0 -10.0 -0.0 10.0 -795.53 0.0 10.0 0.0 -10.175 -10.0 2.628 -10.0 0.0 -10 -true -23 1969-12-31 16:00:12.563 -23.0 cvLH6Eat2yFsyy7p 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -NULL 59 1969-12-31 16:00:00.551 59.0 NULL -59 59 0 59.0 0.0 -59.0 4693.6269999999995 0.0 -59.0 0.0 -10.175 59.0 -0.44542372881355935 59.0 0.0 59 -NULL 5 1969-12-31 16:00:05.034 5.0 NULL -5 5 0 5.0 0.0 -5.0 397.765 0.0 -5.0 0.0 -10.175 5.0 -5.256 5.0 0.0 5 -true 15 1969-12-31 16:00:12.746 15.0 821UdmGbkEf4j -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -true 31 1969-12-31 16:00:00.876 31.0 cvLH6Eat2yFsyy7p -31 31 0 31.0 0.0 -31.0 2466.143 0.0 -31.0 0.0 -10.175 31.0 -0.847741935483871 31.0 0.0 31 -NULL 10 1969-12-31 16:00:15.615 10.0 NULL -10 10 0 10.0 0.0 -10.0 795.53 0.0 -10.0 0.0 -10.175 10.0 -2.628 10.0 0.0 10 -NULL 23 1969-12-31 16:00:12.759 23.0 NULL -23 23 0 23.0 0.0 -23.0 1829.719 0.0 -23.0 0.0 -10.175 23.0 -1.142608695652174 23.0 0.0 23 -NULL -13 1969-12-31 16:00:02.613 -13.0 NULL 13 -13 0 -13.0 -0.0 13.0 -1034.1889999999999 0.0 13.0 0.0 -10.175 -13.0 2.0215384615384617 -13.0 0.0 -13 -true -50 1969-12-31 16:00:02.775 -50.0 821UdmGbkEf4j 50 -50 0 -50.0 -0.0 50.0 -3977.6499999999996 0.0 50.0 0.0 -10.175 -50.0 0.5256000000000001 -50.0 0.0 -50 -true -37 1969-12-31 16:00:08.368 -37.0 cvLH6Eat2yFsyy7p 37 -37 0 -37.0 -0.0 37.0 -2943.461 0.0 37.0 0.0 -10.175 -37.0 0.7102702702702703 -37.0 0.0 -37 -NULL 62 1969-12-31 16:00:14.364 62.0 NULL -62 62 0 62.0 0.0 -62.0 4932.286 0.0 -62.0 0.0 -10.175 62.0 -0.4238709677419355 62.0 0.0 62 -true -21 1969-12-31 16:00:15.712 -21.0 1cGVWH7n1QU 21 -21 0 -21.0 -0.0 21.0 -1670.6129999999998 0.0 21.0 0.0 -10.175 -21.0 1.2514285714285716 -21.0 0.0 -21 -NULL 33 1969-12-31 16:00:15.99 33.0 NULL -33 33 0 33.0 0.0 -33.0 2625.249 0.0 -33.0 0.0 -10.175 33.0 -0.7963636363636364 33.0 0.0 33 -NULL -51 1969-12-31 16:00:03.136 -51.0 NULL 51 -51 0 -51.0 -0.0 51.0 -4057.203 0.0 51.0 0.0 -10.175 -51.0 0.5152941176470588 -51.0 0.0 -51 -NULL 62 1969-12-31 16:00:13.524 62.0 NULL -62 62 0 62.0 0.0 -62.0 4932.286 0.0 -62.0 0.0 -10.175 62.0 -0.4238709677419355 62.0 0.0 62 -NULL 14 1969-12-31 16:00:12.477 14.0 NULL -14 14 0 14.0 0.0 -14.0 1113.742 0.0 -14.0 0.0 -10.175 14.0 -1.8771428571428572 14.0 0.0 14 -NULL -25 1969-12-31 16:00:13.374 -25.0 NULL 25 -25 0 -25.0 -0.0 25.0 -1988.8249999999998 0.0 25.0 0.0 -10.175 -25.0 1.0512000000000001 -25.0 0.0 -25 -true -57 1969-12-31 16:00:08.941 -57.0 cvLH6Eat2yFsyy7p 57 -57 0 -57.0 -0.0 57.0 -4534.521 0.0 57.0 0.0 -10.175 -57.0 0.4610526315789474 -57.0 0.0 -57 -NULL 28 1969-12-31 16:00:12.148 28.0 NULL -28 28 0 28.0 0.0 -28.0 2227.484 0.0 -28.0 0.0 -10.175 28.0 -0.9385714285714286 28.0 0.0 28 -true -54 1969-12-31 16:00:04.345 -54.0 1cGVWH7n1QU 54 -54 0 -54.0 -0.0 54.0 -4295.862 0.0 54.0 0.0 -10.175 -54.0 0.4866666666666667 -54.0 0.0 -54 -NULL 37 1969-12-31 16:00:05.356 37.0 NULL -37 37 0 37.0 0.0 -37.0 2943.461 0.0 -37.0 0.0 -10.175 37.0 -0.7102702702702703 37.0 0.0 37 -NULL -62 1969-12-31 16:00:14.247 -62.0 NULL 62 -62 0 -62.0 -0.0 62.0 -4932.286 0.0 62.0 0.0 -10.175 -62.0 0.4238709677419355 -62.0 0.0 -62 -true -19 1969-12-31 16:00:07.812 -19.0 cvLH6Eat2yFsyy7p 19 -19 0 -19.0 -0.0 19.0 -1511.507 0.0 19.0 0.0 -10.175 -19.0 1.3831578947368421 -19.0 0.0 -19 -true 0 1969-12-31 16:00:10.688 0.0 cvLH6Eat2yFsyy7p 0 0 0 0.0 0.0 -0.0 0.0 0.0 -0.0 0.0 -10.175 0.0 NULL 0.0 NULL 0 -NULL -2 1969-12-31 16:00:00.574 -2.0 NULL 2 -2 0 -2.0 -0.0 2.0 -159.106 0.0 2.0 0.0 -10.175 -2.0 13.14 -2.0 0.0 -2 -true 5 1969-12-31 16:00:03.11 5.0 cvLH6Eat2yFsyy7p -5 5 0 5.0 0.0 -5.0 397.765 0.0 -5.0 0.0 -10.175 5.0 -5.256 5.0 0.0 5 -true -52 1969-12-31 16:00:00.309 -52.0 cvLH6Eat2yFsyy7p 52 -52 0 -52.0 -0.0 52.0 -4136.755999999999 0.0 52.0 0.0 -10.175 -52.0 0.5053846153846154 -52.0 0.0 -52 -true 38 1969-12-31 16:00:14.239 38.0 821UdmGbkEf4j -38 38 0 38.0 0.0 -38.0 3023.014 0.0 -38.0 0.0 -10.175 38.0 -0.6915789473684211 38.0 0.0 38 -true 49 1969-12-31 16:00:05.248 49.0 821UdmGbkEf4j -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 -true 61 1969-12-31 16:00:15.673 61.0 1cGVWH7n1QU -61 61 0 61.0 0.0 -61.0 4852.733 0.0 -61.0 0.0 -10.175 61.0 -0.4308196721311476 61.0 0.0 61 -NULL -36 1969-12-31 16:00:08.615 -36.0 NULL 36 -36 0 -36.0 -0.0 36.0 -2863.908 0.0 36.0 0.0 -10.175 -36.0 0.73 -36.0 0.0 -36 -true 27 1969-12-31 16:00:01.083 27.0 cvLH6Eat2yFsyy7p -27 27 0 27.0 0.0 -27.0 2147.931 0.0 -27.0 0.0 -10.175 27.0 -0.9733333333333334 27.0 0.0 27 -NULL -61 1969-12-31 16:00:03.049 -61.0 NULL 61 -61 0 -61.0 -0.0 61.0 -4852.733 0.0 61.0 0.0 -10.175 -61.0 0.4308196721311476 -61.0 0.0 -61 -true 21 1969-12-31 16:00:08.648 21.0 cvLH6Eat2yFsyy7p -21 21 0 21.0 0.0 -21.0 1670.6129999999998 0.0 -21.0 0.0 -10.175 21.0 -1.2514285714285716 21.0 0.0 21 -true -5 1969-12-31 16:00:01.728 -5.0 1cGVWH7n1QU 5 -5 0 -5.0 -0.0 5.0 -397.765 0.0 5.0 0.0 -10.175 -5.0 5.256 -5.0 0.0 -5 -NULL 11 1969-12-31 16:00:06.603 11.0 NULL -11 11 0 11.0 0.0 -11.0 875.083 0.0 -11.0 0.0 -10.175 11.0 -2.389090909090909 11.0 0.0 11 -NULL -42 1969-12-31 16:00:16.191 -42.0 NULL 42 -42 0 -42.0 -0.0 42.0 -3341.2259999999997 0.0 42.0 0.0 -10.175 -42.0 0.6257142857142858 -42.0 0.0 -42 -true 44 1969-12-31 16:00:06.09 44.0 821UdmGbkEf4j -44 44 0 44.0 0.0 -44.0 3500.332 0.0 -44.0 0.0 -10.175 44.0 -0.5972727272727273 44.0 0.0 44 -NULL 0 1969-12-31 16:00:11.089 0.0 NULL 0 0 0 0.0 0.0 -0.0 0.0 0.0 -0.0 0.0 -10.175 0.0 NULL 0.0 NULL 0 -true -19 1969-12-31 16:00:04.587 -19.0 cvLH6Eat2yFsyy7p 19 -19 0 -19.0 -0.0 19.0 -1511.507 0.0 19.0 0.0 -10.175 -19.0 1.3831578947368421 -19.0 0.0 -19 -true -38 1969-12-31 16:00:07.574 -38.0 cvLH6Eat2yFsyy7p 38 -38 0 -38.0 -0.0 38.0 -3023.014 0.0 38.0 0.0 -10.175 -38.0 0.6915789473684211 -38.0 0.0 -38 -true 18 1969-12-31 16:00:00.259 18.0 cvLH6Eat2yFsyy7p -18 18 0 18.0 0.0 -18.0 1431.954 0.0 -18.0 0.0 -10.175 18.0 -1.46 18.0 0.0 18 -NULL -29 1969-12-31 16:00:09.14 -29.0 NULL 29 -29 0 -29.0 -0.0 29.0 -2307.037 0.0 29.0 0.0 -10.175 -29.0 0.9062068965517242 -29.0 0.0 -29 -NULL 41 1969-12-31 16:00:04.12 41.0 NULL -41 41 0 41.0 0.0 -41.0 3261.673 0.0 -41.0 0.0 -10.175 41.0 -0.6409756097560976 41.0 0.0 41 -NULL 34 1969-12-31 16:00:01.232 34.0 NULL -34 34 0 34.0 0.0 -34.0 2704.8019999999997 0.0 -34.0 0.0 -10.175 34.0 -0.7729411764705882 34.0 0.0 34 -true 23 1969-12-31 16:00:11.343 23.0 cvLH6Eat2yFsyy7p -23 23 0 23.0 0.0 -23.0 1829.719 0.0 -23.0 0.0 -10.175 23.0 -1.142608695652174 23.0 0.0 23 -NULL 49 1969-12-31 16:00:00.148 49.0 NULL -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 -true 17 1969-12-31 16:00:00.229 17.0 1cGVWH7n1QU -17 17 0 17.0 0.0 -17.0 1352.4009999999998 0.0 -17.0 0.0 -10.175 17.0 -1.5458823529411765 17.0 0.0 17 -true 33 1969-12-31 16:00:02.176 33.0 cvLH6Eat2yFsyy7p -33 33 0 33.0 0.0 -33.0 2625.249 0.0 -33.0 0.0 -10.175 33.0 -0.7963636363636364 33.0 0.0 33 -NULL -54 1969-12-31 16:00:08.805 -54.0 NULL 54 -54 0 -54.0 -0.0 54.0 -4295.862 0.0 54.0 0.0 -10.175 -54.0 0.4866666666666667 -54.0 0.0 -54 -NULL 20 1969-12-31 16:00:01.687 20.0 NULL -20 20 0 20.0 0.0 -20.0 1591.06 0.0 -20.0 0.0 -10.175 20.0 -1.314 20.0 0.0 20 -NULL 22 1969-12-31 16:00:05.696 22.0 NULL -22 22 0 22.0 0.0 -22.0 1750.166 0.0 -22.0 0.0 -10.175 22.0 -1.1945454545454546 22.0 0.0 22 -NULL -47 1969-12-31 16:00:10.274 -47.0 NULL 47 -47 0 -47.0 -0.0 47.0 -3738.991 0.0 47.0 0.0 -10.175 -47.0 0.5591489361702128 -47.0 0.0 -47 -true 45 1969-12-31 16:00:03.14 45.0 cvLH6Eat2yFsyy7p -45 45 0 45.0 0.0 -45.0 3579.8849999999998 0.0 -45.0 0.0 -10.175 45.0 -0.5840000000000001 45.0 0.0 45 -NULL 8 1969-12-31 16:00:15.969 8.0 NULL -8 8 0 8.0 0.0 -8.0 636.424 0.0 -8.0 0.0 -10.175 8.0 -3.285 8.0 0.0 8 -true 46 1969-12-31 16:00:31.808 46.0 1cGVWH7n1QU -46 46 0 46.0 0.0 -46.0 3659.438 0.0 -46.0 0.0 -10.175 46.0 -0.571304347826087 46.0 0.0 46 -NULL -56 1969-12-31 16:00:07.212 -56.0 NULL 56 -56 0 -56.0 -0.0 56.0 -4454.968 0.0 56.0 0.0 -10.175 -56.0 0.4692857142857143 -56.0 0.0 -56 -NULL 10 1969-12-31 16:00:12.104 10.0 NULL -10 10 0 10.0 0.0 -10.0 795.53 0.0 -10.0 0.0 -10.175 10.0 -2.628 10.0 0.0 10 -true 13 1969-12-31 16:00:09.642 13.0 cvLH6Eat2yFsyy7p -13 13 0 13.0 0.0 -13.0 1034.1889999999999 0.0 -13.0 0.0 -10.175 13.0 -2.0215384615384617 13.0 0.0 13 -true -49 1969-12-31 16:00:06.919 -49.0 1cGVWH7n1QU 49 -49 0 -49.0 -0.0 49.0 -3898.0969999999998 0.0 49.0 0.0 -10.175 -49.0 0.5363265306122449 -49.0 0.0 -49 -true 48 1969-12-31 16:00:00.923 48.0 1cGVWH7n1QU -48 48 0 48.0 0.0 -48.0 3818.544 0.0 -48.0 0.0 -10.175 48.0 -0.5475 48.0 0.0 48 -NULL 0 1969-12-31 16:00:00.705 0.0 NULL 0 0 0 0.0 0.0 -0.0 0.0 0.0 -0.0 0.0 -10.175 0.0 NULL 0.0 NULL 0 -NULL -31 1969-12-31 16:00:15.012 -31.0 NULL 31 -31 0 -31.0 -0.0 31.0 -2466.143 0.0 31.0 0.0 -10.175 -31.0 0.847741935483871 -31.0 0.0 -31 -NULL 38 1969-12-31 16:00:00.272 38.0 NULL -38 38 0 38.0 0.0 -38.0 3023.014 0.0 -38.0 0.0 -10.175 38.0 -0.6915789473684211 38.0 0.0 38 -true 9 1969-12-31 16:00:12.42 9.0 821UdmGbkEf4j -9 9 0 9.0 0.0 -9.0 715.977 0.0 -9.0 0.0 -10.175 9.0 -2.92 9.0 0.0 9 -NULL 19 1969-12-31 16:00:10.723 19.0 NULL -19 19 0 19.0 0.0 -19.0 1511.507 0.0 -19.0 0.0 -10.175 19.0 -1.3831578947368421 19.0 0.0 19 -true -62 1969-12-31 16:00:00.684 -62.0 cvLH6Eat2yFsyy7p 62 -62 0 -62.0 -0.0 62.0 -4932.286 0.0 62.0 0.0 -10.175 -62.0 0.4238709677419355 -62.0 0.0 -62 -NULL -51 1969-12-31 16:00:08.448 -51.0 NULL 51 -51 0 -51.0 -0.0 51.0 -4057.203 0.0 51.0 0.0 -10.175 -51.0 0.5152941176470588 -51.0 0.0 -51 -NULL 18 1969-12-31 16:00:00.251 18.0 NULL -18 18 0 18.0 0.0 -18.0 1431.954 0.0 -18.0 0.0 -10.175 18.0 -1.46 18.0 0.0 18 -NULL -45 1969-12-31 16:00:16.216 -45.0 NULL 45 -45 0 -45.0 -0.0 45.0 -3579.8849999999998 0.0 45.0 0.0 -10.175 -45.0 0.5840000000000001 -45.0 0.0 -45 -true -18 1969-12-31 16:00:00.368 -18.0 821UdmGbkEf4j 18 -18 0 -18.0 -0.0 18.0 -1431.954 0.0 18.0 0.0 -10.175 -18.0 1.46 -18.0 0.0 -18 -NULL -20 1969-12-31 16:00:00.825 -20.0 NULL 20 -20 0 -20.0 -0.0 20.0 -1591.06 0.0 20.0 0.0 -10.175 -20.0 1.314 -20.0 0.0 -20 -true 13 1969-12-31 16:00:08.799 13.0 1cGVWH7n1QU -13 13 0 13.0 0.0 -13.0 1034.1889999999999 0.0 -13.0 0.0 -10.175 13.0 -2.0215384615384617 13.0 0.0 13 -true 0 1969-12-31 16:00:09.017 0.0 1cGVWH7n1QU 0 0 0 0.0 0.0 -0.0 0.0 0.0 -0.0 0.0 -10.175 0.0 NULL 0.0 NULL 0 -true -60 1969-12-31 16:00:12.223 -60.0 1cGVWH7n1QU 60 -60 0 -60.0 -0.0 60.0 -4773.18 0.0 60.0 0.0 -10.175 -60.0 0.438 -60.0 0.0 -60 -NULL 18 1969-12-31 16:00:05.064 18.0 NULL -18 18 0 18.0 0.0 -18.0 1431.954 0.0 -18.0 0.0 -10.175 18.0 -1.46 18.0 0.0 18 -true -59 1969-12-31 16:00:00.648 -59.0 821UdmGbkEf4j 59 -59 0 -59.0 -0.0 59.0 -4693.6269999999995 0.0 59.0 0.0 -10.175 -59.0 0.44542372881355935 -59.0 0.0 -59 -NULL 7 1969-12-31 16:00:09.539 7.0 NULL -7 7 0 7.0 0.0 -7.0 556.871 0.0 -7.0 0.0 -10.175 7.0 -3.7542857142857144 7.0 0.0 7 -true 59 1969-12-31 16:00:00.636 59.0 821UdmGbkEf4j -59 59 0 59.0 0.0 -59.0 4693.6269999999995 0.0 -59.0 0.0 -10.175 59.0 -0.44542372881355935 59.0 0.0 59 -NULL 16 1969-12-31 16:00:13.869 16.0 NULL -16 16 0 16.0 0.0 -16.0 1272.848 0.0 -16.0 0.0 -10.175 16.0 -1.6425 16.0 0.0 16 -true -11 1969-12-31 16:00:13.832 -11.0 1cGVWH7n1QU 11 -11 0 -11.0 -0.0 11.0 -875.083 0.0 11.0 0.0 -10.175 -11.0 2.389090909090909 -11.0 0.0 -11 -NULL 13 1969-12-31 16:00:05.028 13.0 NULL -13 13 0 13.0 0.0 -13.0 1034.1889999999999 0.0 -13.0 0.0 -10.175 13.0 -2.0215384615384617 13.0 0.0 13 -NULL 15 1969-12-31 16:00:13.735 15.0 NULL -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -true 10 1969-12-31 16:00:12.05 10.0 cvLH6Eat2yFsyy7p -10 10 0 10.0 0.0 -10.0 795.53 0.0 -10.0 0.0 -10.175 10.0 -2.628 10.0 0.0 10 -true -26 1969-12-31 16:00:15.49 -26.0 1cGVWH7n1QU 26 -26 0 -26.0 -0.0 26.0 -2068.3779999999997 0.0 26.0 0.0 -10.175 -26.0 1.0107692307692309 -26.0 0.0 -26 -true 43 1969-12-31 16:00:06.114 43.0 cvLH6Eat2yFsyy7p -43 43 0 43.0 0.0 -43.0 3420.779 0.0 -43.0 0.0 -10.175 43.0 -0.6111627906976744 43.0 0.0 43 -NULL 21 1969-12-31 16:00:09.07 21.0 NULL -21 21 0 21.0 0.0 -21.0 1670.6129999999998 0.0 -21.0 0.0 -10.175 21.0 -1.2514285714285716 21.0 0.0 21 -NULL 17 1969-12-31 16:00:00.197 17.0 NULL -17 17 0 17.0 0.0 -17.0 1352.4009999999998 0.0 -17.0 0.0 -10.175 17.0 -1.5458823529411765 17.0 0.0 17 -true -30 1969-12-31 16:00:16.146 -30.0 821UdmGbkEf4j 30 -30 0 -30.0 -0.0 30.0 -2386.59 0.0 30.0 0.0 -10.175 -30.0 0.876 -30.0 0.0 -30 -true -15 1969-12-31 16:00:11.044 -15.0 821UdmGbkEf4j 15 -15 0 -15.0 -0.0 15.0 -1193.295 0.0 15.0 0.0 -10.175 -15.0 1.752 -15.0 0.0 -15 -NULL 16 1969-12-31 16:00:02.221 16.0 NULL -16 16 0 16.0 0.0 -16.0 1272.848 0.0 -16.0 0.0 -10.175 16.0 -1.6425 16.0 0.0 16 -true -39 1969-12-31 16:00:01.29 -39.0 1cGVWH7n1QU 39 -39 0 -39.0 -0.0 39.0 -3102.567 0.0 39.0 0.0 -10.175 -39.0 0.6738461538461539 -39.0 0.0 -39 -NULL 31 1969-12-31 16:00:06.838 31.0 NULL -31 31 0 31.0 0.0 -31.0 2466.143 0.0 -31.0 0.0 -10.175 31.0 -0.847741935483871 31.0 0.0 31 -NULL 61 1969-12-31 16:00:13.868 61.0 NULL -61 61 0 61.0 0.0 -61.0 4852.733 0.0 -61.0 0.0 -10.175 61.0 -0.4308196721311476 61.0 0.0 61 -NULL 59 1969-12-31 16:00:12.325 59.0 NULL -59 59 0 59.0 0.0 -59.0 4693.6269999999995 0.0 -59.0 0.0 -10.175 59.0 -0.44542372881355935 59.0 0.0 59 -NULL 53 1969-12-31 16:00:02.966 53.0 NULL -53 53 0 53.0 0.0 -53.0 4216.309 0.0 -53.0 0.0 -10.175 53.0 -0.4958490566037736 53.0 0.0 53 -true 38 1969-12-31 16:00:09.169 38.0 cvLH6Eat2yFsyy7p -38 38 0 38.0 0.0 -38.0 3023.014 0.0 -38.0 0.0 -10.175 38.0 -0.6915789473684211 38.0 0.0 38 -NULL 23 1969-12-31 16:00:11.785 23.0 NULL -23 23 0 23.0 0.0 -23.0 1829.719 0.0 -23.0 0.0 -10.175 23.0 -1.142608695652174 23.0 0.0 23 -NULL -30 1969-12-31 16:00:11.15 -30.0 NULL 30 -30 0 -30.0 -0.0 30.0 -2386.59 0.0 30.0 0.0 -10.175 -30.0 0.876 -30.0 0.0 -30 -true 32 1969-12-31 16:00:03.538 32.0 821UdmGbkEf4j -32 32 0 32.0 0.0 -32.0 2545.696 0.0 -32.0 0.0 -10.175 32.0 -0.82125 32.0 0.0 32 -NULL 36 1969-12-31 16:00:05.251 36.0 NULL -36 36 0 36.0 0.0 -36.0 2863.908 0.0 -36.0 0.0 -10.175 36.0 -0.73 36.0 0.0 36 -NULL -45 1969-12-31 16:00:07.705 -45.0 NULL 45 -45 0 -45.0 -0.0 45.0 -3579.8849999999998 0.0 45.0 0.0 -10.175 -45.0 0.5840000000000001 -45.0 0.0 -45 -NULL 61 1969-12-31 16:00:12.2 61.0 NULL -61 61 0 61.0 0.0 -61.0 4852.733 0.0 -61.0 0.0 -10.175 61.0 -0.4308196721311476 61.0 0.0 61 -NULL -23 1969-12-31 16:00:01.644 -23.0 NULL 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -NULL -9 1969-12-31 16:00:11.08 -9.0 NULL 9 -9 0 -9.0 -0.0 9.0 -715.977 0.0 9.0 0.0 -10.175 -9.0 2.92 -9.0 0.0 -9 -NULL 35 1969-12-31 16:00:00.249 35.0 NULL -35 35 0 35.0 0.0 -35.0 2784.355 0.0 -35.0 0.0 -10.175 35.0 -0.7508571428571429 35.0 0.0 35 -NULL -28 1969-12-31 16:00:09.93 -28.0 NULL 28 -28 0 -28.0 -0.0 28.0 -2227.484 0.0 28.0 0.0 -10.175 -28.0 0.9385714285714286 -28.0 0.0 -28 -NULL -6 1969-12-31 16:00:02.928 -6.0 NULL 6 -6 0 -6.0 -0.0 6.0 -477.318 0.0 6.0 0.0 -10.175 -6.0 4.38 -6.0 0.0 -6 -NULL 32 1969-12-31 16:00:02.445 32.0 NULL -32 32 0 32.0 0.0 -32.0 2545.696 0.0 -32.0 0.0 -10.175 32.0 -0.82125 32.0 0.0 32 -NULL 45 1969-12-31 16:00:05.29 45.0 NULL -45 45 0 45.0 0.0 -45.0 3579.8849999999998 0.0 -45.0 0.0 -10.175 45.0 -0.5840000000000001 45.0 0.0 45 -NULL 26 1969-12-31 16:00:03.857 26.0 NULL -26 26 0 26.0 0.0 -26.0 2068.3779999999997 0.0 -26.0 0.0 -10.175 26.0 -1.0107692307692309 26.0 0.0 26 -true -10 1969-12-31 16:00:08.676 -10.0 cvLH6Eat2yFsyy7p 10 -10 0 -10.0 -0.0 10.0 -795.53 0.0 10.0 0.0 -10.175 -10.0 2.628 -10.0 0.0 -10 -true 23 1969-12-31 16:00:12.66 23.0 cvLH6Eat2yFsyy7p -23 23 0 23.0 0.0 -23.0 1829.719 0.0 -23.0 0.0 -10.175 23.0 -1.142608695652174 23.0 0.0 23 -true 45 1969-12-31 16:00:01.31 45.0 821UdmGbkEf4j -45 45 0 45.0 0.0 -45.0 3579.8849999999998 0.0 -45.0 0.0 -10.175 45.0 -0.5840000000000001 45.0 0.0 45 -true 17 1969-12-31 16:00:00.863 17.0 1cGVWH7n1QU -17 17 0 17.0 0.0 -17.0 1352.4009999999998 0.0 -17.0 0.0 -10.175 17.0 -1.5458823529411765 17.0 0.0 17 -true -34 1969-12-31 16:00:08.47 -34.0 1cGVWH7n1QU 34 -34 0 -34.0 -0.0 34.0 -2704.8019999999997 0.0 34.0 0.0 -10.175 -34.0 0.7729411764705882 -34.0 0.0 -34 -true -35 1969-12-31 16:00:04.001 -35.0 1cGVWH7n1QU 35 -35 0 -35.0 -0.0 35.0 -2784.355 0.0 35.0 0.0 -10.175 -35.0 0.7508571428571429 -35.0 0.0 -35 -true -26 1969-12-31 16:00:07.287 -26.0 821UdmGbkEf4j 26 -26 0 -26.0 -0.0 26.0 -2068.3779999999997 0.0 26.0 0.0 -10.175 -26.0 1.0107692307692309 -26.0 0.0 -26 -true 25 1969-12-31 16:00:00.533 25.0 cvLH6Eat2yFsyy7p -25 25 0 25.0 0.0 -25.0 1988.8249999999998 0.0 -25.0 0.0 -10.175 25.0 -1.0512000000000001 25.0 0.0 25 -NULL -45 1969-12-31 16:00:01.592 -45.0 NULL 45 -45 0 -45.0 -0.0 45.0 -3579.8849999999998 0.0 45.0 0.0 -10.175 -45.0 0.5840000000000001 -45.0 0.0 -45 -true -52 1969-12-31 16:00:03.914 -52.0 1cGVWH7n1QU 52 -52 0 -52.0 -0.0 52.0 -4136.755999999999 0.0 52.0 0.0 -10.175 -52.0 0.5053846153846154 -52.0 0.0 -52 -true -14 1969-12-31 16:00:11.533 -14.0 1cGVWH7n1QU 14 -14 0 -14.0 -0.0 14.0 -1113.742 0.0 14.0 0.0 -10.175 -14.0 1.8771428571428572 -14.0 0.0 -14 -true 26 1969-12-31 16:00:03.945 26.0 cvLH6Eat2yFsyy7p -26 26 0 26.0 0.0 -26.0 2068.3779999999997 0.0 -26.0 0.0 -10.175 26.0 -1.0107692307692309 26.0 0.0 26 -NULL -6 1969-12-31 16:00:14.697 -6.0 NULL 6 -6 0 -6.0 -0.0 6.0 -477.318 0.0 6.0 0.0 -10.175 -6.0 4.38 -6.0 0.0 -6 -NULL -43 1969-12-31 16:00:13.212 -43.0 NULL 43 -43 0 -43.0 -0.0 43.0 -3420.779 0.0 43.0 0.0 -10.175 -43.0 0.6111627906976744 -43.0 0.0 -43 -true 60 1969-12-31 16:00:12.437 60.0 1cGVWH7n1QU -60 60 0 60.0 0.0 -60.0 4773.18 0.0 -60.0 0.0 -10.175 60.0 -0.438 60.0 0.0 60 -true 42 1969-12-31 16:00:06.575 42.0 1cGVWH7n1QU -42 42 0 42.0 0.0 -42.0 3341.2259999999997 0.0 -42.0 0.0 -10.175 42.0 -0.6257142857142858 42.0 0.0 42 -true -42 1969-12-31 16:00:05.941 -42.0 821UdmGbkEf4j 42 -42 0 -42.0 -0.0 42.0 -3341.2259999999997 0.0 42.0 0.0 -10.175 -42.0 0.6257142857142858 -42.0 0.0 -42 -NULL 12 1969-12-31 16:00:08.779 12.0 NULL -12 12 0 12.0 0.0 -12.0 954.636 0.0 -12.0 0.0 -10.175 12.0 -2.19 12.0 0.0 12 -NULL 11 1969-12-31 16:00:14.747 11.0 NULL -11 11 0 11.0 0.0 -11.0 875.083 0.0 -11.0 0.0 -10.175 11.0 -2.389090909090909 11.0 0.0 11 -NULL 20 1969-12-31 16:00:06.482 20.0 NULL -20 20 0 20.0 0.0 -20.0 1591.06 0.0 -20.0 0.0 -10.175 20.0 -1.314 20.0 0.0 20 -true 13 1969-12-31 16:00:00.029 13.0 1cGVWH7n1QU -13 13 0 13.0 0.0 -13.0 1034.1889999999999 0.0 -13.0 0.0 -10.175 13.0 -2.0215384615384617 13.0 0.0 13 -true -36 1969-12-31 16:00:00.355 -36.0 1cGVWH7n1QU 36 -36 0 -36.0 -0.0 36.0 -2863.908 0.0 36.0 0.0 -10.175 -36.0 0.73 -36.0 0.0 -36 -NULL 55 1969-12-31 16:00:08.191 55.0 NULL -55 55 0 55.0 0.0 -55.0 4375.415 0.0 -55.0 0.0 -10.175 55.0 -0.47781818181818186 55.0 0.0 55 -true 11 1969-12-31 16:00:06.062 11.0 cvLH6Eat2yFsyy7p -11 11 0 11.0 0.0 -11.0 875.083 0.0 -11.0 0.0 -10.175 11.0 -2.389090909090909 11.0 0.0 11 -true -56 1969-12-31 16:00:08.784 -56.0 821UdmGbkEf4j 56 -56 0 -56.0 -0.0 56.0 -4454.968 0.0 56.0 0.0 -10.175 -56.0 0.4692857142857143 -56.0 0.0 -56 -true 26 1969-12-31 16:00:06.115 26.0 1cGVWH7n1QU -26 26 0 26.0 0.0 -26.0 2068.3779999999997 0.0 -26.0 0.0 -10.175 26.0 -1.0107692307692309 26.0 0.0 26 -NULL -14 1969-12-31 16:00:10.884 -14.0 NULL 14 -14 0 -14.0 -0.0 14.0 -1113.742 0.0 14.0 0.0 -10.175 -14.0 1.8771428571428572 -14.0 0.0 -14 -NULL -36 1969-12-31 16:00:04.964 -36.0 NULL 36 -36 0 -36.0 -0.0 36.0 -2863.908 0.0 36.0 0.0 -10.175 -36.0 0.73 -36.0 0.0 -36 -NULL 0 1969-12-31 16:00:12.049 0.0 NULL 0 0 0 0.0 0.0 -0.0 0.0 0.0 -0.0 0.0 -10.175 0.0 NULL 0.0 NULL 0 -true 18 1969-12-31 16:00:02.986 18.0 821UdmGbkEf4j -18 18 0 18.0 0.0 -18.0 1431.954 0.0 -18.0 0.0 -10.175 18.0 -1.46 18.0 0.0 18 -NULL 41 1969-12-31 16:00:01.652 41.0 NULL -41 41 0 41.0 0.0 -41.0 3261.673 0.0 -41.0 0.0 -10.175 41.0 -0.6409756097560976 41.0 0.0 41 -NULL -50 1969-12-31 16:00:10.867 -50.0 NULL 50 -50 0 -50.0 -0.0 50.0 -3977.6499999999996 0.0 50.0 0.0 -10.175 -50.0 0.5256000000000001 -50.0 0.0 -50 -true 56 1969-12-31 16:00:05.898 56.0 821UdmGbkEf4j -56 56 0 56.0 0.0 -56.0 4454.968 0.0 -56.0 0.0 -10.175 56.0 -0.4692857142857143 56.0 0.0 56 -true -30 1969-12-31 16:00:03.7 -30.0 1cGVWH7n1QU 30 -30 0 -30.0 -0.0 30.0 -2386.59 0.0 30.0 0.0 -10.175 -30.0 0.876 -30.0 0.0 -30 -true -58 1969-12-31 16:00:08.661 -58.0 cvLH6Eat2yFsyy7p 58 -58 0 -58.0 -0.0 58.0 -4614.074 0.0 58.0 0.0 -10.175 -58.0 0.4531034482758621 -58.0 0.0 -58 -true -9 1969-12-31 16:00:05.687 -9.0 821UdmGbkEf4j 9 -9 0 -9.0 -0.0 9.0 -715.977 0.0 9.0 0.0 -10.175 -9.0 2.92 -9.0 0.0 -9 -true 4 1969-12-31 16:00:06.941 4.0 cvLH6Eat2yFsyy7p -4 4 0 4.0 0.0 -4.0 318.212 0.0 -4.0 0.0 -10.175 4.0 -6.57 4.0 0.0 4 -NULL -37 1969-12-31 16:00:02.112 -37.0 NULL 37 -37 0 -37.0 -0.0 37.0 -2943.461 0.0 37.0 0.0 -10.175 -37.0 0.7102702702702703 -37.0 0.0 -37 -NULL 17 1969-12-31 16:00:10.738 17.0 NULL -17 17 0 17.0 0.0 -17.0 1352.4009999999998 0.0 -17.0 0.0 -10.175 17.0 -1.5458823529411765 17.0 0.0 17 -true -23 1969-12-31 16:00:15.855 -23.0 1cGVWH7n1QU 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -true 31 1969-12-31 16:00:12.208 31.0 cvLH6Eat2yFsyy7p -31 31 0 31.0 0.0 -31.0 2466.143 0.0 -31.0 0.0 -10.175 31.0 -0.847741935483871 31.0 0.0 31 -true 1 1969-12-31 16:00:01.798 1.0 1cGVWH7n1QU -1 1 0 1.0 0.0 -1.0 79.553 0.0 -1.0 0.0 -10.175 1.0 -26.28 1.0 0.0 1 -true 19 1969-12-31 16:00:11.092 19.0 1cGVWH7n1QU -19 19 0 19.0 0.0 -19.0 1511.507 0.0 -19.0 0.0 -10.175 19.0 -1.3831578947368421 19.0 0.0 19 -true -23 1969-12-31 16:00:14.625 -23.0 cvLH6Eat2yFsyy7p 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -NULL -10 1969-12-31 16:00:13.684 -10.0 NULL 10 -10 0 -10.0 -0.0 10.0 -795.53 0.0 10.0 0.0 -10.175 -10.0 2.628 -10.0 0.0 -10 -NULL -6 1969-12-31 16:00:00.472 -6.0 NULL 6 -6 0 -6.0 -0.0 6.0 -477.318 0.0 6.0 0.0 -10.175 -6.0 4.38 -6.0 0.0 -6 -true 13 1969-12-31 16:00:09.923 13.0 1cGVWH7n1QU -13 13 0 13.0 0.0 -13.0 1034.1889999999999 0.0 -13.0 0.0 -10.175 13.0 -2.0215384615384617 13.0 0.0 13 -NULL -31 1969-12-31 16:00:16.059 -31.0 NULL 31 -31 0 -31.0 -0.0 31.0 -2466.143 0.0 31.0 0.0 -10.175 -31.0 0.847741935483871 -31.0 0.0 -31 -NULL -41 1969-12-31 16:00:00.282 -41.0 NULL 41 -41 0 -41.0 -0.0 41.0 -3261.673 0.0 41.0 0.0 -10.175 -41.0 0.6409756097560976 -41.0 0.0 -41 -true -21 1969-12-31 16:00:10.284 -21.0 cvLH6Eat2yFsyy7p 21 -21 0 -21.0 -0.0 21.0 -1670.6129999999998 0.0 21.0 0.0 -10.175 -21.0 1.2514285714285716 -21.0 0.0 -21 -true 57 1969-12-31 16:00:00.652 57.0 821UdmGbkEf4j -57 57 0 57.0 0.0 -57.0 4534.521 0.0 -57.0 0.0 -10.175 57.0 -0.4610526315789474 57.0 0.0 57 -true 28 1969-12-31 16:00:03.856 28.0 cvLH6Eat2yFsyy7p -28 28 0 28.0 0.0 -28.0 2227.484 0.0 -28.0 0.0 -10.175 28.0 -0.9385714285714286 28.0 0.0 28 -NULL 46 1969-12-31 16:00:09.752 46.0 NULL -46 46 0 46.0 0.0 -46.0 3659.438 0.0 -46.0 0.0 -10.175 46.0 -0.571304347826087 46.0 0.0 46 -NULL -62 1969-12-31 16:00:01.734 -62.0 NULL 62 -62 0 -62.0 -0.0 62.0 -4932.286 0.0 62.0 0.0 -10.175 -62.0 0.4238709677419355 -62.0 0.0 -62 -true -21 1969-12-31 16:00:12.522 -21.0 1cGVWH7n1QU 21 -21 0 -21.0 -0.0 21.0 -1670.6129999999998 0.0 21.0 0.0 -10.175 -21.0 1.2514285714285716 -21.0 0.0 -21 -NULL -8 1969-12-31 16:00:06.828 -8.0 NULL 8 -8 0 -8.0 -0.0 8.0 -636.424 0.0 8.0 0.0 -10.175 -8.0 3.285 -8.0 0.0 -8 -NULL -23 1969-12-31 16:00:03.539 -23.0 NULL 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -NULL -50 1969-12-31 16:00:04.696 -50.0 NULL 50 -50 0 -50.0 -0.0 50.0 -3977.6499999999996 0.0 50.0 0.0 -10.175 -50.0 0.5256000000000001 -50.0 0.0 -50 -true -36 1969-12-31 16:00:00.913 -36.0 1cGVWH7n1QU 36 -36 0 -36.0 -0.0 36.0 -2863.908 0.0 36.0 0.0 -10.175 -36.0 0.73 -36.0 0.0 -36 -true -38 1969-12-31 16:00:14.751 -38.0 821UdmGbkEf4j 38 -38 0 -38.0 -0.0 38.0 -3023.014 0.0 38.0 0.0 -10.175 -38.0 0.6915789473684211 -38.0 0.0 -38 -true -2 1969-12-31 16:00:15.431 -2.0 1cGVWH7n1QU 2 -2 0 -2.0 -0.0 2.0 -159.106 0.0 2.0 0.0 -10.175 -2.0 13.14 -2.0 0.0 -2 -true 5 1969-12-31 16:00:02.011 5.0 821UdmGbkEf4j -5 5 0 5.0 0.0 -5.0 397.765 0.0 -5.0 0.0 -10.175 5.0 -5.256 5.0 0.0 5 -NULL -15 1969-12-31 16:00:03.261 -15.0 NULL 15 -15 0 -15.0 -0.0 15.0 -1193.295 0.0 15.0 0.0 -10.175 -15.0 1.752 -15.0 0.0 -15 -true -24 1969-12-31 16:00:11.168 -24.0 821UdmGbkEf4j 24 -24 0 -24.0 -0.0 24.0 -1909.272 0.0 24.0 0.0 -10.175 -24.0 1.095 -24.0 0.0 -24 -true 59 1969-12-31 16:00:10.903 59.0 821UdmGbkEf4j -59 59 0 59.0 0.0 -59.0 4693.6269999999995 0.0 -59.0 0.0 -10.175 59.0 -0.44542372881355935 59.0 0.0 59 -NULL -37 1969-12-31 16:00:11.241 -37.0 NULL 37 -37 0 -37.0 -0.0 37.0 -2943.461 0.0 37.0 0.0 -10.175 -37.0 0.7102702702702703 -37.0 0.0 -37 -NULL -64 1969-12-31 16:00:11.912 -64.0 NULL 64 -64 0 -64.0 -0.0 64.0 -5091.392 0.0 64.0 0.0 -10.175 -64.0 0.410625 -64.0 0.0 -64 -NULL 12 1969-12-31 16:00:15.321 12.0 NULL -12 12 0 12.0 0.0 -12.0 954.636 0.0 -12.0 0.0 -10.175 12.0 -2.19 12.0 0.0 12 -NULL -19 1969-12-31 16:00:01.923 -19.0 NULL 19 -19 0 -19.0 -0.0 19.0 -1511.507 0.0 19.0 0.0 -10.175 -19.0 1.3831578947368421 -19.0 0.0 -19 -true -56 1969-12-31 16:00:10.802 -56.0 821UdmGbkEf4j 56 -56 0 -56.0 -0.0 56.0 -4454.968 0.0 56.0 0.0 -10.175 -56.0 0.4692857142857143 -56.0 0.0 -56 -NULL -63 1969-12-31 16:00:03.552 -63.0 NULL 63 -63 0 -63.0 -0.0 63.0 -5011.839 0.0 63.0 0.0 -10.175 -63.0 0.41714285714285715 -63.0 0.0 -63 -true 61 1969-12-31 16:00:07.514 61.0 1cGVWH7n1QU -61 61 0 61.0 0.0 -61.0 4852.733 0.0 -61.0 0.0 -10.175 61.0 -0.4308196721311476 61.0 0.0 61 -NULL -4 1969-12-31 16:00:12.848 -4.0 NULL 4 -4 0 -4.0 -0.0 4.0 -318.212 0.0 4.0 0.0 -10.175 -4.0 6.57 -4.0 0.0 -4 -NULL 48 1969-12-31 16:00:09.582 48.0 NULL -48 48 0 48.0 0.0 -48.0 3818.544 0.0 -48.0 0.0 -10.175 48.0 -0.5475 48.0 0.0 48 -NULL 1 1969-12-31 16:00:02.579 1.0 NULL -1 1 0 1.0 0.0 -1.0 79.553 0.0 -1.0 0.0 -10.175 1.0 -26.28 1.0 0.0 1 -NULL -37 1969-12-31 16:00:16.202 -37.0 NULL 37 -37 0 -37.0 -0.0 37.0 -2943.461 0.0 37.0 0.0 -10.175 -37.0 0.7102702702702703 -37.0 0.0 -37 -NULL -28 1969-12-31 16:00:04.747 -28.0 NULL 28 -28 0 -28.0 -0.0 28.0 -2227.484 0.0 28.0 0.0 -10.175 -28.0 0.9385714285714286 -28.0 0.0 -28 -NULL 32 1969-12-31 16:00:14.026 32.0 NULL -32 32 0 32.0 0.0 -32.0 2545.696 0.0 -32.0 0.0 -10.175 32.0 -0.82125 32.0 0.0 32 -true -4 1969-12-31 16:00:00.032 -4.0 cvLH6Eat2yFsyy7p 4 -4 0 -4.0 -0.0 4.0 -318.212 0.0 4.0 0.0 -10.175 -4.0 6.57 -4.0 0.0 -4 -NULL 39 1969-12-31 16:00:01.122 39.0 NULL -39 39 0 39.0 0.0 -39.0 3102.567 0.0 -39.0 0.0 -10.175 39.0 -0.6738461538461539 39.0 0.0 39 -NULL 2 1969-12-31 16:00:09.161 2.0 NULL -2 2 0 2.0 0.0 -2.0 159.106 0.0 -2.0 0.0 -10.175 2.0 -13.14 2.0 0.0 2 -true -20 1969-12-31 16:00:13.893 -20.0 821UdmGbkEf4j 20 -20 0 -20.0 -0.0 20.0 -1591.06 0.0 20.0 0.0 -10.175 -20.0 1.314 -20.0 0.0 -20 -true -29 1969-12-31 16:00:00.709 -29.0 1cGVWH7n1QU 29 -29 0 -29.0 -0.0 29.0 -2307.037 0.0 29.0 0.0 -10.175 -29.0 0.9062068965517242 -29.0 0.0 -29 -true 32 1969-12-31 16:00:03.761 32.0 1cGVWH7n1QU -32 32 0 32.0 0.0 -32.0 2545.696 0.0 -32.0 0.0 -10.175 32.0 -0.82125 32.0 0.0 32 -true -48 1969-12-31 16:00:00.329 -48.0 821UdmGbkEf4j 48 -48 0 -48.0 -0.0 48.0 -3818.544 0.0 48.0 0.0 -10.175 -48.0 0.5475 -48.0 0.0 -48 -NULL -37 1969-12-31 16:00:09.88 -37.0 NULL 37 -37 0 -37.0 -0.0 37.0 -2943.461 0.0 37.0 0.0 -10.175 -37.0 0.7102702702702703 -37.0 0.0 -37 -true -1 1969-12-31 16:00:05.785 -1.0 821UdmGbkEf4j 1 -1 0 -1.0 -0.0 1.0 -79.553 0.0 1.0 0.0 -10.175 -1.0 26.28 -1.0 0.0 -1 -NULL 4 1969-12-31 16:00:00.119 4.0 NULL -4 4 0 4.0 0.0 -4.0 318.212 0.0 -4.0 0.0 -10.175 4.0 -6.57 4.0 0.0 4 -true 58 1969-12-31 16:00:07.29 58.0 cvLH6Eat2yFsyy7p -58 58 0 58.0 0.0 -58.0 4614.074 0.0 -58.0 0.0 -10.175 58.0 -0.4531034482758621 58.0 0.0 58 -true 9 1969-12-31 16:00:15.747 9.0 1cGVWH7n1QU -9 9 0 9.0 0.0 -9.0 715.977 0.0 -9.0 0.0 -10.175 9.0 -2.92 9.0 0.0 9 -NULL 42 1969-12-31 16:00:11.591 42.0 NULL -42 42 0 42.0 0.0 -42.0 3341.2259999999997 0.0 -42.0 0.0 -10.175 42.0 -0.6257142857142858 42.0 0.0 42 -NULL -7 1969-12-31 16:00:12.641 -7.0 NULL 7 -7 0 -7.0 -0.0 7.0 -556.871 0.0 7.0 0.0 -10.175 -7.0 3.7542857142857144 -7.0 0.0 -7 -true -59 1969-12-31 16:00:15.278 -59.0 cvLH6Eat2yFsyy7p 59 -59 0 -59.0 -0.0 59.0 -4693.6269999999995 0.0 59.0 0.0 -10.175 -59.0 0.44542372881355935 -59.0 0.0 -59 -NULL 62 1969-12-31 16:00:02.308 62.0 NULL -62 62 0 62.0 0.0 -62.0 4932.286 0.0 -62.0 0.0 -10.175 62.0 -0.4238709677419355 62.0 0.0 62 -true -56 1969-12-31 16:00:13.534 -56.0 1cGVWH7n1QU 56 -56 0 -56.0 -0.0 56.0 -4454.968 0.0 56.0 0.0 -10.175 -56.0 0.4692857142857143 -56.0 0.0 -56 -true -55 1969-12-31 16:00:16.055 -55.0 cvLH6Eat2yFsyy7p 55 -55 0 -55.0 -0.0 55.0 -4375.415 0.0 55.0 0.0 -10.175 -55.0 0.47781818181818186 -55.0 0.0 -55 -NULL -13 1969-12-31 16:00:03.134 -13.0 NULL 13 -13 0 -13.0 -0.0 13.0 -1034.1889999999999 0.0 13.0 0.0 -10.175 -13.0 2.0215384615384617 -13.0 0.0 -13 -true -16 1969-12-31 16:00:10.725 -16.0 cvLH6Eat2yFsyy7p 16 -16 0 -16.0 -0.0 16.0 -1272.848 0.0 16.0 0.0 -10.175 -16.0 1.6425 -16.0 0.0 -16 -NULL 4 1969-12-31 16:00:14.894 4.0 NULL -4 4 0 4.0 0.0 -4.0 318.212 0.0 -4.0 0.0 -10.175 4.0 -6.57 4.0 0.0 4 -NULL 42 1969-12-31 16:00:07.179 42.0 NULL -42 42 0 42.0 0.0 -42.0 3341.2259999999997 0.0 -42.0 0.0 -10.175 42.0 -0.6257142857142858 42.0 0.0 42 -NULL -6 1969-12-31 16:00:04.136 -6.0 NULL 6 -6 0 -6.0 -0.0 6.0 -477.318 0.0 6.0 0.0 -10.175 -6.0 4.38 -6.0 0.0 -6 -true -32 1969-12-31 16:00:09.05 -32.0 cvLH6Eat2yFsyy7p 32 -32 0 -32.0 -0.0 32.0 -2545.696 0.0 32.0 0.0 -10.175 -32.0 0.82125 -32.0 0.0 -32 -NULL 21 1969-12-31 16:00:02.931 21.0 NULL -21 21 0 21.0 0.0 -21.0 1670.6129999999998 0.0 -21.0 0.0 -10.175 21.0 -1.2514285714285716 21.0 0.0 21 -NULL 17 1969-12-31 16:00:11.758 17.0 NULL -17 17 0 17.0 0.0 -17.0 1352.4009999999998 0.0 -17.0 0.0 -10.175 17.0 -1.5458823529411765 17.0 0.0 17 -true -32 1969-12-31 16:00:00.779 -32.0 1cGVWH7n1QU 32 -32 0 -32.0 -0.0 32.0 -2545.696 0.0 32.0 0.0 -10.175 -32.0 0.82125 -32.0 0.0 -32 -NULL 11 1969-12-31 16:00:07.9 11.0 NULL -11 11 0 11.0 0.0 -11.0 875.083 0.0 -11.0 0.0 -10.175 11.0 -2.389090909090909 11.0 0.0 11 -true 33 1969-12-31 16:00:31.808 79.553 cvLH6Eat2yFsyy7p -33 33 0 159.1060028076172 0.0 -159.1060028076172 6328.679920677185 0.0 -159.1060028076172 0.0 -10.175 159.1060028076172 -0.16517290068418367 79.553 0.0 33 -true 53 1969-12-31 16:00:11.362 53.0 1cGVWH7n1QU -53 53 0 53.0 0.0 -53.0 4216.309 0.0 -53.0 0.0 -10.175 53.0 -0.4958490566037736 53.0 0.0 53 -true 14 1969-12-31 16:00:03.911 14.0 cvLH6Eat2yFsyy7p -14 14 0 14.0 0.0 -14.0 1113.742 0.0 -14.0 0.0 -10.175 14.0 -1.8771428571428572 14.0 0.0 14 -true 51 1969-12-31 16:00:05.78 51.0 821UdmGbkEf4j -51 51 0 51.0 0.0 -51.0 4057.203 0.0 -51.0 0.0 -10.175 51.0 -0.5152941176470588 51.0 0.0 51 -NULL 29 1969-12-31 16:00:13.093 29.0 NULL -29 29 0 29.0 0.0 -29.0 2307.037 0.0 -29.0 0.0 -10.175 29.0 -0.9062068965517242 29.0 0.0 29 -NULL -60 1969-12-31 16:00:02.742 -60.0 NULL 60 -60 0 -60.0 -0.0 60.0 -4773.18 0.0 60.0 0.0 -10.175 -60.0 0.438 -60.0 0.0 -60 -true -43 1969-12-31 16:00:08.695 -43.0 cvLH6Eat2yFsyy7p 43 -43 0 -43.0 -0.0 43.0 -3420.779 0.0 43.0 0.0 -10.175 -43.0 0.6111627906976744 -43.0 0.0 -43 -true -52 1969-12-31 16:00:10.928 -52.0 1cGVWH7n1QU 52 -52 0 -52.0 -0.0 52.0 -4136.755999999999 0.0 52.0 0.0 -10.175 -52.0 0.5053846153846154 -52.0 0.0 -52 -true 30 1969-12-31 16:00:04.647 30.0 cvLH6Eat2yFsyy7p -30 30 0 30.0 0.0 -30.0 2386.59 0.0 -30.0 0.0 -10.175 30.0 -0.876 30.0 0.0 30 -true 53 1969-12-31 16:00:04.181 53.0 cvLH6Eat2yFsyy7p -53 53 0 53.0 0.0 -53.0 4216.309 0.0 -53.0 0.0 -10.175 53.0 -0.4958490566037736 53.0 0.0 53 -NULL -27 1969-12-31 16:00:10.289 -27.0 NULL 27 -27 0 -27.0 -0.0 27.0 -2147.931 0.0 27.0 0.0 -10.175 -27.0 0.9733333333333334 -27.0 0.0 -27 -true -29 1969-12-31 16:00:04.151 -29.0 821UdmGbkEf4j 29 -29 0 -29.0 -0.0 29.0 -2307.037 0.0 29.0 0.0 -10.175 -29.0 0.9062068965517242 -29.0 0.0 -29 -NULL -13 1969-12-31 16:00:02.495 -13.0 NULL 13 -13 0 -13.0 -0.0 13.0 -1034.1889999999999 0.0 13.0 0.0 -10.175 -13.0 2.0215384615384617 -13.0 0.0 -13 -true 31 1969-12-31 16:00:14.388 31.0 821UdmGbkEf4j -31 31 0 31.0 0.0 -31.0 2466.143 0.0 -31.0 0.0 -10.175 31.0 -0.847741935483871 31.0 0.0 31 -NULL -27 1969-12-31 16:00:03.127 -27.0 NULL 27 -27 0 -27.0 -0.0 27.0 -2147.931 0.0 27.0 0.0 -10.175 -27.0 0.9733333333333334 -27.0 0.0 -27 -NULL 39 1969-12-31 16:00:10.575 39.0 NULL -39 39 0 39.0 0.0 -39.0 3102.567 0.0 -39.0 0.0 -10.175 39.0 -0.6738461538461539 39.0 0.0 39 -true -60 1969-12-31 16:00:07.838 -60.0 1cGVWH7n1QU 60 -60 0 -60.0 -0.0 60.0 -4773.18 0.0 60.0 0.0 -10.175 -60.0 0.438 -60.0 0.0 -60 -NULL 10 1969-12-31 16:00:12.322 10.0 NULL -10 10 0 10.0 0.0 -10.0 795.53 0.0 -10.0 0.0 -10.175 10.0 -2.628 10.0 0.0 10 -NULL -37 1969-12-31 16:00:05.391 -37.0 NULL 37 -37 0 -37.0 -0.0 37.0 -2943.461 0.0 37.0 0.0 -10.175 -37.0 0.7102702702702703 -37.0 0.0 -37 -NULL -36 1969-12-31 16:00:06.661 -36.0 NULL 36 -36 0 -36.0 -0.0 36.0 -2863.908 0.0 36.0 0.0 -10.175 -36.0 0.73 -36.0 0.0 -36 -NULL 4 1969-12-31 16:00:12.291 4.0 NULL -4 4 0 4.0 0.0 -4.0 318.212 0.0 -4.0 0.0 -10.175 4.0 -6.57 4.0 0.0 4 -true -64 1969-12-31 16:00:00.013 -64.0 1cGVWH7n1QU 64 -64 0 -64.0 -0.0 64.0 -5091.392 0.0 64.0 0.0 -10.175 -64.0 0.410625 -64.0 0.0 -64 -NULL 24 1969-12-31 16:00:02.543 24.0 NULL -24 24 0 24.0 0.0 -24.0 1909.272 0.0 -24.0 0.0 -10.175 24.0 -1.095 24.0 0.0 24 -NULL 34 1969-12-31 16:00:00.82 34.0 NULL -34 34 0 34.0 0.0 -34.0 2704.8019999999997 0.0 -34.0 0.0 -10.175 34.0 -0.7729411764705882 34.0 0.0 34 -true -37 1969-12-31 16:00:11.953 -37.0 821UdmGbkEf4j 37 -37 0 -37.0 -0.0 37.0 -2943.461 0.0 37.0 0.0 -10.175 -37.0 0.7102702702702703 -37.0 0.0 -37 -true 24 1969-12-31 16:00:02.541 24.0 cvLH6Eat2yFsyy7p -24 24 0 24.0 0.0 -24.0 1909.272 0.0 -24.0 0.0 -10.175 24.0 -1.095 24.0 0.0 24 -true 2 1969-12-31 16:00:06.546 2.0 821UdmGbkEf4j -2 2 0 2.0 0.0 -2.0 159.106 0.0 -2.0 0.0 -10.175 2.0 -13.14 2.0 0.0 2 -true 25 1969-12-31 16:00:09.765 25.0 1cGVWH7n1QU -25 25 0 25.0 0.0 -25.0 1988.8249999999998 0.0 -25.0 0.0 -10.175 25.0 -1.0512000000000001 25.0 0.0 25 -NULL -57 1969-12-31 16:00:02.12 -57.0 NULL 57 -57 0 -57.0 -0.0 57.0 -4534.521 0.0 57.0 0.0 -10.175 -57.0 0.4610526315789474 -57.0 0.0 -57 -true 6 1969-12-31 16:00:05.885 6.0 821UdmGbkEf4j -6 6 0 6.0 0.0 -6.0 477.318 0.0 -6.0 0.0 -10.175 6.0 -4.38 6.0 0.0 6 -true -57 1969-12-31 16:00:01.322 -57.0 1cGVWH7n1QU 57 -57 0 -57.0 -0.0 57.0 -4534.521 0.0 57.0 0.0 -10.175 -57.0 0.4610526315789474 -57.0 0.0 -57 -true -15 1969-12-31 16:00:04.37 -15.0 821UdmGbkEf4j 15 -15 0 -15.0 -0.0 15.0 -1193.295 0.0 15.0 0.0 -10.175 -15.0 1.752 -15.0 0.0 -15 -NULL 61 1969-12-31 16:00:09.251 61.0 NULL -61 61 0 61.0 0.0 -61.0 4852.733 0.0 -61.0 0.0 -10.175 61.0 -0.4308196721311476 61.0 0.0 61 -true -50 1969-12-31 16:00:01.151 -50.0 821UdmGbkEf4j 50 -50 0 -50.0 -0.0 50.0 -3977.6499999999996 0.0 50.0 0.0 -10.175 -50.0 0.5256000000000001 -50.0 0.0 -50 -NULL 30 1969-12-31 16:00:08.142 30.0 NULL -30 30 0 30.0 0.0 -30.0 2386.59 0.0 -30.0 0.0 -10.175 30.0 -0.876 30.0 0.0 30 -NULL -23 1969-12-31 16:00:13.118 -26.28 NULL 23 -23 0 -26.280000686645508 -0.0 26.280000686645508 -2090.65289462471 0.0 26.280000686645508 0.0 -10.175 -26.280000686645508 0.9999999738719373 -26.28 0.0 -23 -true 49 1969-12-31 16:00:15.541 49.0 cvLH6Eat2yFsyy7p -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 -NULL 24 1969-12-31 16:00:14.524 24.0 NULL -24 24 0 24.0 0.0 -24.0 1909.272 0.0 -24.0 0.0 -10.175 24.0 -1.095 24.0 0.0 24 -true -57 1969-12-31 16:00:10.994 -57.0 1cGVWH7n1QU 57 -57 0 -57.0 -0.0 57.0 -4534.521 0.0 57.0 0.0 -10.175 -57.0 0.4610526315789474 -57.0 0.0 -57 -NULL -22 1969-12-31 16:00:04.494 -22.0 NULL 22 -22 0 -22.0 -0.0 22.0 -1750.166 0.0 22.0 0.0 -10.175 -22.0 1.1945454545454546 -22.0 0.0 -22 -NULL -41 1969-12-31 16:00:07.461 -41.0 NULL 41 -41 0 -41.0 -0.0 41.0 -3261.673 0.0 41.0 0.0 -10.175 -41.0 0.6409756097560976 -41.0 0.0 -41 -true -29 1969-12-31 16:00:11.804 -29.0 1cGVWH7n1QU 29 -29 0 -29.0 -0.0 29.0 -2307.037 0.0 29.0 0.0 -10.175 -29.0 0.9062068965517242 -29.0 0.0 -29 -true 1 1969-12-31 16:00:01.191 1.0 1cGVWH7n1QU -1 1 0 1.0 0.0 -1.0 79.553 0.0 -1.0 0.0 -10.175 1.0 -26.28 1.0 0.0 1 -NULL -23 1969-12-31 16:00:14.071 -23.0 NULL 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -true -1 1969-12-31 16:00:08.104 -1.0 821UdmGbkEf4j 1 -1 0 -1.0 -0.0 1.0 -79.553 0.0 1.0 0.0 -10.175 -1.0 26.28 -1.0 0.0 -1 -true -8 1969-12-31 16:00:09.987 -8.0 cvLH6Eat2yFsyy7p 8 -8 0 -8.0 -0.0 8.0 -636.424 0.0 8.0 0.0 -10.175 -8.0 3.285 -8.0 0.0 -8 -NULL 22 1969-12-31 16:00:00.767 22.0 NULL -22 22 0 22.0 0.0 -22.0 1750.166 0.0 -22.0 0.0 -10.175 22.0 -1.1945454545454546 22.0 0.0 22 -true 15 1969-12-31 16:00:10.576 15.0 821UdmGbkEf4j -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -true 37 1969-12-31 16:00:10.807 37.0 cvLH6Eat2yFsyy7p -37 37 0 37.0 0.0 -37.0 2943.461 0.0 -37.0 0.0 -10.175 37.0 -0.7102702702702703 37.0 0.0 37 -NULL -45 1969-12-31 16:00:08.351 -45.0 NULL 45 -45 0 -45.0 -0.0 45.0 -3579.8849999999998 0.0 45.0 0.0 -10.175 -45.0 0.5840000000000001 -45.0 0.0 -45 -NULL 45 1969-12-31 16:00:01.005 45.0 NULL -45 45 0 45.0 0.0 -45.0 3579.8849999999998 0.0 -45.0 0.0 -10.175 45.0 -0.5840000000000001 45.0 0.0 45 -true -22 1969-12-31 16:00:12.136 -22.0 821UdmGbkEf4j 22 -22 0 -22.0 -0.0 22.0 -1750.166 0.0 22.0 0.0 -10.175 -22.0 1.1945454545454546 -22.0 0.0 -22 -true 40 1969-12-31 16:00:02.842 40.0 1cGVWH7n1QU -40 40 0 40.0 0.0 -40.0 3182.12 0.0 -40.0 0.0 -10.175 40.0 -0.657 40.0 0.0 40 -true -32 1969-12-31 16:00:00.995 -32.0 cvLH6Eat2yFsyy7p 32 -32 0 -32.0 -0.0 32.0 -2545.696 0.0 32.0 0.0 -10.175 -32.0 0.82125 -32.0 0.0 -32 -true -34 1969-12-31 16:00:00.008 -34.0 cvLH6Eat2yFsyy7p 34 -34 0 -34.0 -0.0 34.0 -2704.8019999999997 0.0 34.0 0.0 -10.175 -34.0 0.7729411764705882 -34.0 0.0 -34 -NULL 33 1969-12-31 16:00:04.631 33.0 NULL -33 33 0 33.0 0.0 -33.0 2625.249 0.0 -33.0 0.0 -10.175 33.0 -0.7963636363636364 33.0 0.0 33 -NULL 44 1969-12-31 16:00:10.163 44.0 NULL -44 44 0 44.0 0.0 -44.0 3500.332 0.0 -44.0 0.0 -10.175 44.0 -0.5972727272727273 44.0 0.0 44 -NULL 55 1969-12-31 16:00:00.805 55.0 NULL -55 55 0 55.0 0.0 -55.0 4375.415 0.0 -55.0 0.0 -10.175 55.0 -0.47781818181818186 55.0 0.0 55 -true -46 1969-12-31 16:00:06.837 -46.0 1cGVWH7n1QU 46 -46 0 -46.0 -0.0 46.0 -3659.438 0.0 46.0 0.0 -10.175 -46.0 0.571304347826087 -46.0 0.0 -46 -NULL 8 1969-12-31 16:00:09.519 8.0 NULL -8 8 0 8.0 0.0 -8.0 636.424 0.0 -8.0 0.0 -10.175 8.0 -3.285 8.0 0.0 8 -true 9 1969-12-31 16:00:00.706 9.0 1cGVWH7n1QU -9 9 0 9.0 0.0 -9.0 715.977 0.0 -9.0 0.0 -10.175 9.0 -2.92 9.0 0.0 9 -NULL 51 1969-12-31 16:00:15.668 51.0 NULL -51 51 0 51.0 0.0 -51.0 4057.203 0.0 -51.0 0.0 -10.175 51.0 -0.5152941176470588 51.0 0.0 51 -NULL 9 1969-12-31 16:00:13.686 9.0 NULL -9 9 0 9.0 0.0 -9.0 715.977 0.0 -9.0 0.0 -10.175 9.0 -2.92 9.0 0.0 9 -NULL 35 1969-12-31 16:00:08.37 35.0 NULL -35 35 0 35.0 0.0 -35.0 2784.355 0.0 -35.0 0.0 -10.175 35.0 -0.7508571428571429 35.0 0.0 35 -true -19 1969-12-31 16:00:15.416 -19.0 cvLH6Eat2yFsyy7p 19 -19 0 -19.0 -0.0 19.0 -1511.507 0.0 19.0 0.0 -10.175 -19.0 1.3831578947368421 -19.0 0.0 -19 -true 5 1969-12-31 16:00:05.085 5.0 cvLH6Eat2yFsyy7p -5 5 0 5.0 0.0 -5.0 397.765 0.0 -5.0 0.0 -10.175 5.0 -5.256 5.0 0.0 5 -NULL 44 1969-12-31 16:00:14.098 44.0 NULL -44 44 0 44.0 0.0 -44.0 3500.332 0.0 -44.0 0.0 -10.175 44.0 -0.5972727272727273 44.0 0.0 44 -NULL 35 1969-12-31 16:00:06.051 35.0 NULL -35 35 0 35.0 0.0 -35.0 2784.355 0.0 -35.0 0.0 -10.175 35.0 -0.7508571428571429 35.0 0.0 35 -true 50 1969-12-31 16:00:03.897 50.0 cvLH6Eat2yFsyy7p -50 50 0 50.0 0.0 -50.0 3977.6499999999996 0.0 -50.0 0.0 -10.175 50.0 -0.5256000000000001 50.0 0.0 50 -NULL -35 1969-12-31 16:00:11.847 -35.0 NULL 35 -35 0 -35.0 -0.0 35.0 -2784.355 0.0 35.0 0.0 -10.175 -35.0 0.7508571428571429 -35.0 0.0 -35 -true 26 1969-12-31 16:00:02.824 26.0 821UdmGbkEf4j -26 26 0 26.0 0.0 -26.0 2068.3779999999997 0.0 -26.0 0.0 -10.175 26.0 -1.0107692307692309 26.0 0.0 26 -true 56 1969-12-31 16:00:00.762 56.0 cvLH6Eat2yFsyy7p -56 56 0 56.0 0.0 -56.0 4454.968 0.0 -56.0 0.0 -10.175 56.0 -0.4692857142857143 56.0 0.0 56 -true 42 1969-12-31 16:00:00.302 42.0 1cGVWH7n1QU -42 42 0 42.0 0.0 -42.0 3341.2259999999997 0.0 -42.0 0.0 -10.175 42.0 -0.6257142857142858 42.0 0.0 42 -NULL -22 1969-12-31 16:00:13.052 -22.0 NULL 22 -22 0 -22.0 -0.0 22.0 -1750.166 0.0 22.0 0.0 -10.175 -22.0 1.1945454545454546 -22.0 0.0 -22 -NULL -13 1969-12-31 16:00:14.436 -13.0 NULL 13 -13 0 -13.0 -0.0 13.0 -1034.1889999999999 0.0 13.0 0.0 -10.175 -13.0 2.0215384615384617 -13.0 0.0 -13 -NULL 34 1969-12-31 16:00:09.791 34.0 NULL -34 34 0 34.0 0.0 -34.0 2704.8019999999997 0.0 -34.0 0.0 -10.175 34.0 -0.7729411764705882 34.0 0.0 34 -true -22 1969-12-31 16:00:09.476 -22.0 821UdmGbkEf4j 22 -22 0 -22.0 -0.0 22.0 -1750.166 0.0 22.0 0.0 -10.175 -22.0 1.1945454545454546 -22.0 0.0 -22 -NULL 13 1969-12-31 16:00:12.498 13.0 NULL -13 13 0 13.0 0.0 -13.0 1034.1889999999999 0.0 -13.0 0.0 -10.175 13.0 -2.0215384615384617 13.0 0.0 13 -true 17 1969-12-31 16:00:00.283 17.0 821UdmGbkEf4j -17 17 0 17.0 0.0 -17.0 1352.4009999999998 0.0 -17.0 0.0 -10.175 17.0 -1.5458823529411765 17.0 0.0 17 -true 10 1969-12-31 16:00:00.55 10.0 1cGVWH7n1QU -10 10 0 10.0 0.0 -10.0 795.53 0.0 -10.0 0.0 -10.175 10.0 -2.628 10.0 0.0 10 -NULL -17 1969-12-31 16:00:09.528 -17.0 NULL 17 -17 0 -17.0 -0.0 17.0 -1352.4009999999998 0.0 17.0 0.0 -10.175 -17.0 1.5458823529411765 -17.0 0.0 -17 -true -51 1969-12-31 16:00:00.447 -51.0 cvLH6Eat2yFsyy7p 51 -51 0 -51.0 -0.0 51.0 -4057.203 0.0 51.0 0.0 -10.175 -51.0 0.5152941176470588 -51.0 0.0 -51 -true 8 1969-12-31 16:00:00.335 8.0 cvLH6Eat2yFsyy7p -8 8 0 8.0 0.0 -8.0 636.424 0.0 -8.0 0.0 -10.175 8.0 -3.285 8.0 0.0 8 -true 31 1969-12-31 16:00:04.388 31.0 cvLH6Eat2yFsyy7p -31 31 0 31.0 0.0 -31.0 2466.143 0.0 -31.0 0.0 -10.175 31.0 -0.847741935483871 31.0 0.0 31 -true 12 1969-12-31 16:00:13.265 12.0 1cGVWH7n1QU -12 12 0 12.0 0.0 -12.0 954.636 0.0 -12.0 0.0 -10.175 12.0 -2.19 12.0 0.0 12 -true 50 1969-12-31 16:00:00.099 50.0 cvLH6Eat2yFsyy7p -50 50 0 50.0 0.0 -50.0 3977.6499999999996 0.0 -50.0 0.0 -10.175 50.0 -0.5256000000000001 50.0 0.0 50 -NULL -23 1969-12-31 16:00:10.192 -26.28 NULL 23 -23 0 -26.280000686645508 -0.0 26.280000686645508 -2090.65289462471 0.0 26.280000686645508 0.0 -10.175 -26.280000686645508 0.9999999738719373 -26.28 0.0 -23 -NULL -46 1969-12-31 16:00:00.013 -46.0 NULL 46 -46 0 -46.0 -0.0 46.0 -3659.438 0.0 46.0 0.0 -10.175 -46.0 0.571304347826087 -46.0 0.0 -46 -NULL -20 1969-12-31 16:00:13.612 -20.0 NULL 20 -20 0 -20.0 -0.0 20.0 -1591.06 0.0 20.0 0.0 -10.175 -20.0 1.314 -20.0 0.0 -20 -true 25 1969-12-31 16:00:03.508 25.0 1cGVWH7n1QU -25 25 0 25.0 0.0 -25.0 1988.8249999999998 0.0 -25.0 0.0 -10.175 25.0 -1.0512000000000001 25.0 0.0 25 -NULL -58 1969-12-31 16:00:08.74 -58.0 NULL 58 -58 0 -58.0 -0.0 58.0 -4614.074 0.0 58.0 0.0 -10.175 -58.0 0.4531034482758621 -58.0 0.0 -58 -true -5 1969-12-31 16:00:07.276 -5.0 821UdmGbkEf4j 5 -5 0 -5.0 -0.0 5.0 -397.765 0.0 5.0 0.0 -10.175 -5.0 5.256 -5.0 0.0 -5 -NULL -48 1969-12-31 16:00:08.693 -48.0 NULL 48 -48 0 -48.0 -0.0 48.0 -3818.544 0.0 48.0 0.0 -10.175 -48.0 0.5475 -48.0 0.0 -48 -true -1 1969-12-31 16:00:05.451 -1.0 821UdmGbkEf4j 1 -1 0 -1.0 -0.0 1.0 -79.553 0.0 1.0 0.0 -10.175 -1.0 26.28 -1.0 0.0 -1 -true 43 1969-12-31 16:00:12.781 43.0 821UdmGbkEf4j -43 43 0 43.0 0.0 -43.0 3420.779 0.0 -43.0 0.0 -10.175 43.0 -0.6111627906976744 43.0 0.0 43 -true 12 1969-12-31 16:00:09.759 12.0 cvLH6Eat2yFsyy7p -12 12 0 12.0 0.0 -12.0 954.636 0.0 -12.0 0.0 -10.175 12.0 -2.19 12.0 0.0 12 -NULL 51 1969-12-31 16:00:02.415 51.0 NULL -51 51 0 51.0 0.0 -51.0 4057.203 0.0 -51.0 0.0 -10.175 51.0 -0.5152941176470588 51.0 0.0 51 -true -21 1969-12-31 16:00:07.599 -21.0 821UdmGbkEf4j 21 -21 0 -21.0 -0.0 21.0 -1670.6129999999998 0.0 21.0 0.0 -10.175 -21.0 1.2514285714285716 -21.0 0.0 -21 -NULL -29 1969-12-31 16:00:03.855 -29.0 NULL 29 -29 0 -29.0 -0.0 29.0 -2307.037 0.0 29.0 0.0 -10.175 -29.0 0.9062068965517242 -29.0 0.0 -29 -NULL -63 1969-12-31 16:00:06.852 -63.0 NULL 63 -63 0 -63.0 -0.0 63.0 -5011.839 0.0 63.0 0.0 -10.175 -63.0 0.41714285714285715 -63.0 0.0 -63 -true -33 1969-12-31 16:00:08.579 -33.0 1cGVWH7n1QU 33 -33 0 -33.0 -0.0 33.0 -2625.249 0.0 33.0 0.0 -10.175 -33.0 0.7963636363636364 -33.0 0.0 -33 -true 18 1969-12-31 16:00:13.767 18.0 1cGVWH7n1QU -18 18 0 18.0 0.0 -18.0 1431.954 0.0 -18.0 0.0 -10.175 18.0 -1.46 18.0 0.0 18 -true 22 1969-12-31 16:00:16.163 22.0 1cGVWH7n1QU -22 22 0 22.0 0.0 -22.0 1750.166 0.0 -22.0 0.0 -10.175 22.0 -1.1945454545454546 22.0 0.0 22 -true 33 1969-12-31 16:00:09.801 33.0 821UdmGbkEf4j -33 33 0 33.0 0.0 -33.0 2625.249 0.0 -33.0 0.0 -10.175 33.0 -0.7963636363636364 33.0 0.0 33 -NULL -45 1969-12-31 16:00:10.036 -45.0 NULL 45 -45 0 -45.0 -0.0 45.0 -3579.8849999999998 0.0 45.0 0.0 -10.175 -45.0 0.5840000000000001 -45.0 0.0 -45 -true 52 1969-12-31 16:00:10.592 52.0 821UdmGbkEf4j -52 52 0 52.0 0.0 -52.0 4136.755999999999 0.0 -52.0 0.0 -10.175 52.0 -0.5053846153846154 52.0 0.0 52 -true 61 1969-12-31 16:00:13.741 61.0 821UdmGbkEf4j -61 61 0 61.0 0.0 -61.0 4852.733 0.0 -61.0 0.0 -10.175 61.0 -0.4308196721311476 61.0 0.0 61 -NULL -53 1969-12-31 16:00:00.446 -53.0 NULL 53 -53 0 -53.0 -0.0 53.0 -4216.309 0.0 53.0 0.0 -10.175 -53.0 0.4958490566037736 -53.0 0.0 -53 -true -62 1969-12-31 16:00:15.153 -62.0 821UdmGbkEf4j 62 -62 0 -62.0 -0.0 62.0 -4932.286 0.0 62.0 0.0 -10.175 -62.0 0.4238709677419355 -62.0 0.0 -62 -NULL 10 1969-12-31 16:00:07.365 10.0 NULL -10 10 0 10.0 0.0 -10.0 795.53 0.0 -10.0 0.0 -10.175 10.0 -2.628 10.0 0.0 10 -NULL 5 1969-12-31 16:00:11.083 5.0 NULL -5 5 0 5.0 0.0 -5.0 397.765 0.0 -5.0 0.0 -10.175 5.0 -5.256 5.0 0.0 5 -true 10 1969-12-31 16:00:08.135 10.0 1cGVWH7n1QU -10 10 0 10.0 0.0 -10.0 795.53 0.0 -10.0 0.0 -10.175 10.0 -2.628 10.0 0.0 10 -true 48 1969-12-31 16:00:01.05 48.0 821UdmGbkEf4j -48 48 0 48.0 0.0 -48.0 3818.544 0.0 -48.0 0.0 -10.175 48.0 -0.5475 48.0 0.0 48 -NULL -28 1969-12-31 16:00:12.581 -28.0 NULL 28 -28 0 -28.0 -0.0 28.0 -2227.484 0.0 28.0 0.0 -10.175 -28.0 0.9385714285714286 -28.0 0.0 -28 -true -10 1969-12-31 16:00:10.058 -10.0 821UdmGbkEf4j 10 -10 0 -10.0 -0.0 10.0 -795.53 0.0 10.0 0.0 -10.175 -10.0 2.628 -10.0 0.0 -10 -NULL -50 1969-12-31 16:00:14.175 -50.0 NULL 50 -50 0 -50.0 -0.0 50.0 -3977.6499999999996 0.0 50.0 0.0 -10.175 -50.0 0.5256000000000001 -50.0 0.0 -50 -NULL -28 1969-12-31 16:00:09.589 -28.0 NULL 28 -28 0 -28.0 -0.0 28.0 -2227.484 0.0 28.0 0.0 -10.175 -28.0 0.9385714285714286 -28.0 0.0 -28 -NULL -6 1969-12-31 16:00:09.101 -6.0 NULL 6 -6 0 -6.0 -0.0 6.0 -477.318 0.0 6.0 0.0 -10.175 -6.0 4.38 -6.0 0.0 -6 -true 4 1969-12-31 16:00:16.558 4.0 1cGVWH7n1QU -4 4 0 4.0 0.0 -4.0 318.212 0.0 -4.0 0.0 -10.175 4.0 -6.57 4.0 0.0 4 -NULL 48 1969-12-31 16:00:11.476 48.0 NULL -48 48 0 48.0 0.0 -48.0 3818.544 0.0 -48.0 0.0 -10.175 48.0 -0.5475 48.0 0.0 48 -NULL 32 1969-12-31 16:00:07.675 32.0 NULL -32 32 0 32.0 0.0 -32.0 2545.696 0.0 -32.0 0.0 -10.175 32.0 -0.82125 32.0 0.0 32 -true 3 1969-12-31 16:00:05.406 3.0 821UdmGbkEf4j -3 3 0 3.0 0.0 -3.0 238.659 0.0 -3.0 0.0 -10.175 3.0 -8.76 3.0 0.0 3 -NULL -39 1969-12-31 16:00:13.144 -39.0 NULL 39 -39 0 -39.0 -0.0 39.0 -3102.567 0.0 39.0 0.0 -10.175 -39.0 0.6738461538461539 -39.0 0.0 -39 -true -53 1969-12-31 16:00:00.234 -53.0 821UdmGbkEf4j 53 -53 0 -53.0 -0.0 53.0 -4216.309 0.0 53.0 0.0 -10.175 -53.0 0.4958490566037736 -53.0 0.0 -53 -NULL -13 1969-12-31 16:00:05.24 -13.0 NULL 13 -13 0 -13.0 -0.0 13.0 -1034.1889999999999 0.0 13.0 0.0 -10.175 -13.0 2.0215384615384617 -13.0 0.0 -13 -true -42 1969-12-31 16:00:03.248 -42.0 cvLH6Eat2yFsyy7p 42 -42 0 -42.0 -0.0 42.0 -3341.2259999999997 0.0 42.0 0.0 -10.175 -42.0 0.6257142857142858 -42.0 0.0 -42 -NULL -39 1969-12-31 16:00:02.014 -39.0 NULL 39 -39 0 -39.0 -0.0 39.0 -3102.567 0.0 39.0 0.0 -10.175 -39.0 0.6738461538461539 -39.0 0.0 -39 -NULL -29 1969-12-31 16:00:01.125 -29.0 NULL 29 -29 0 -29.0 -0.0 29.0 -2307.037 0.0 29.0 0.0 -10.175 -29.0 0.9062068965517242 -29.0 0.0 -29 -NULL -15 1969-12-31 16:00:13.438 -15.0 NULL 15 -15 0 -15.0 -0.0 15.0 -1193.295 0.0 15.0 0.0 -10.175 -15.0 1.752 -15.0 0.0 -15 -NULL 62 1969-12-31 16:00:05.027 62.0 NULL -62 62 0 62.0 0.0 -62.0 4932.286 0.0 -62.0 0.0 -10.175 62.0 -0.4238709677419355 62.0 0.0 62 -true -9 1969-12-31 16:00:13.814 -9.0 1cGVWH7n1QU 9 -9 0 -9.0 -0.0 9.0 -715.977 0.0 9.0 0.0 -10.175 -9.0 2.92 -9.0 0.0 -9 -NULL -40 1969-12-31 16:00:03.741 -40.0 NULL 40 -40 0 -40.0 -0.0 40.0 -3182.12 0.0 40.0 0.0 -10.175 -40.0 0.657 -40.0 0.0 -40 -NULL 7 1969-12-31 16:00:01.055 7.0 NULL -7 7 0 7.0 0.0 -7.0 556.871 0.0 -7.0 0.0 -10.175 7.0 -3.7542857142857144 7.0 0.0 7 -true -11 1969-12-31 16:00:00.639 -11.0 cvLH6Eat2yFsyy7p 11 -11 0 -11.0 -0.0 11.0 -875.083 0.0 11.0 0.0 -10.175 -11.0 2.389090909090909 -11.0 0.0 -11 -true 14 1969-12-31 16:00:11.528 14.0 821UdmGbkEf4j -14 14 0 14.0 0.0 -14.0 1113.742 0.0 -14.0 0.0 -10.175 14.0 -1.8771428571428572 14.0 0.0 14 -true 46 1969-12-31 16:00:05.507 46.0 821UdmGbkEf4j -46 46 0 46.0 0.0 -46.0 3659.438 0.0 -46.0 0.0 -10.175 46.0 -0.571304347826087 46.0 0.0 46 -true -8 1969-12-31 16:00:02.109 -8.0 cvLH6Eat2yFsyy7p 8 -8 0 -8.0 -0.0 8.0 -636.424 0.0 8.0 0.0 -10.175 -8.0 3.285 -8.0 0.0 -8 -NULL -50 1969-12-31 16:00:15.707 -50.0 NULL 50 -50 0 -50.0 -0.0 50.0 -3977.6499999999996 0.0 50.0 0.0 -10.175 -50.0 0.5256000000000001 -50.0 0.0 -50 -true -54 1969-12-31 16:00:05.201 -54.0 cvLH6Eat2yFsyy7p 54 -54 0 -54.0 -0.0 54.0 -4295.862 0.0 54.0 0.0 -10.175 -54.0 0.4866666666666667 -54.0 0.0 -54 -NULL -18 1969-12-31 16:00:07.986 -18.0 NULL 18 -18 0 -18.0 -0.0 18.0 -1431.954 0.0 18.0 0.0 -10.175 -18.0 1.46 -18.0 0.0 -18 -NULL 34 1969-12-31 16:00:02.403 34.0 NULL -34 34 0 34.0 0.0 -34.0 2704.8019999999997 0.0 -34.0 0.0 -10.175 34.0 -0.7729411764705882 34.0 0.0 34 -NULL -34 1969-12-31 16:00:01.562 -34.0 NULL 34 -34 0 -34.0 -0.0 34.0 -2704.8019999999997 0.0 34.0 0.0 -10.175 -34.0 0.7729411764705882 -34.0 0.0 -34 -NULL 15 1969-12-31 16:00:00.838 15.0 NULL -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -NULL 32 1969-12-31 16:00:02.305 32.0 NULL -32 32 0 32.0 0.0 -32.0 2545.696 0.0 -32.0 0.0 -10.175 32.0 -0.82125 32.0 0.0 32 -NULL 40 1969-12-31 16:00:01.103 40.0 NULL -40 40 0 40.0 0.0 -40.0 3182.12 0.0 -40.0 0.0 -10.175 40.0 -0.657 40.0 0.0 40 -NULL 60 1969-12-31 16:00:02.526 60.0 NULL -60 60 0 60.0 0.0 -60.0 4773.18 0.0 -60.0 0.0 -10.175 60.0 -0.438 60.0 0.0 60 -true -9 1969-12-31 16:00:10.522 -9.0 821UdmGbkEf4j 9 -9 0 -9.0 -0.0 9.0 -715.977 0.0 9.0 0.0 -10.175 -9.0 2.92 -9.0 0.0 -9 -true 33 1969-12-31 16:00:12.176 33.0 821UdmGbkEf4j -33 33 0 33.0 0.0 -33.0 2625.249 0.0 -33.0 0.0 -10.175 33.0 -0.7963636363636364 33.0 0.0 33 -NULL 33 1969-12-31 16:00:15.142 79.553 NULL -33 33 0 79.5530014038086 0.0 -79.5530014038086 6328.679920677185 0.0 -79.5530014038086 0.0 -10.175 79.5530014038086 -0.33034580136836733 79.553 0.0 33 -NULL -14 1969-12-31 16:00:11.065 -14.0 NULL 14 -14 0 -14.0 -0.0 14.0 -1113.742 0.0 14.0 0.0 -10.175 -14.0 1.8771428571428572 -14.0 0.0 -14 -NULL -21 1969-12-31 16:00:04.229 -21.0 NULL 21 -21 0 -21.0 -0.0 21.0 -1670.6129999999998 0.0 21.0 0.0 -10.175 -21.0 1.2514285714285716 -21.0 0.0 -21 -true -46 1969-12-31 16:00:13.166 -46.0 1cGVWH7n1QU 46 -46 0 -46.0 -0.0 46.0 -3659.438 0.0 46.0 0.0 -10.175 -46.0 0.571304347826087 -46.0 0.0 -46 -NULL 17 1969-12-31 16:00:00.284 10.175 NULL -17 17 0 10.175000190734863 0.0 -10.175000190734863 809.4517901735305 0.0 -10.175000190734863 0.0 -10.175 10.175000190734863 -2.58280093438524 10.175 0.0 17 -NULL -18 1969-12-31 16:00:16.054 -18.0 NULL 18 -18 0 -18.0 -0.0 18.0 -1431.954 0.0 18.0 0.0 -10.175 -18.0 1.46 -18.0 0.0 -18 -true -43 1969-12-31 16:00:05.025 -43.0 821UdmGbkEf4j 43 -43 0 -43.0 -0.0 43.0 -3420.779 0.0 43.0 0.0 -10.175 -43.0 0.6111627906976744 -43.0 0.0 -43 -NULL 3 1969-12-31 16:00:06.132 3.0 NULL -3 3 0 3.0 0.0 -3.0 238.659 0.0 -3.0 0.0 -10.175 3.0 -8.76 3.0 0.0 3 -true 18 1969-12-31 16:00:10.553 18.0 821UdmGbkEf4j -18 18 0 18.0 0.0 -18.0 1431.954 0.0 -18.0 0.0 -10.175 18.0 -1.46 18.0 0.0 18 -NULL 38 1969-12-31 16:00:11.028 38.0 NULL -38 38 0 38.0 0.0 -38.0 3023.014 0.0 -38.0 0.0 -10.175 38.0 -0.6915789473684211 38.0 0.0 38 -true -56 1969-12-31 16:00:08.296 -56.0 cvLH6Eat2yFsyy7p 56 -56 0 -56.0 -0.0 56.0 -4454.968 0.0 56.0 0.0 -10.175 -56.0 0.4692857142857143 -56.0 0.0 -56 -NULL 1 1969-12-31 16:00:00.519 1.0 NULL -1 1 0 1.0 0.0 -1.0 79.553 0.0 -1.0 0.0 -10.175 1.0 -26.28 1.0 0.0 1 -NULL 27 1969-12-31 16:00:15.909 27.0 NULL -27 27 0 27.0 0.0 -27.0 2147.931 0.0 -27.0 0.0 -10.175 27.0 -0.9733333333333334 27.0 0.0 27 -true 57 1969-12-31 16:00:16.278 57.0 821UdmGbkEf4j -57 57 0 57.0 0.0 -57.0 4534.521 0.0 -57.0 0.0 -10.175 57.0 -0.4610526315789474 57.0 0.0 57 -true 6 1969-12-31 16:00:02.078 6.0 cvLH6Eat2yFsyy7p -6 6 0 6.0 0.0 -6.0 477.318 0.0 -6.0 0.0 -10.175 6.0 -4.38 6.0 0.0 6 -true 22 1969-12-31 16:00:13.395 22.0 1cGVWH7n1QU -22 22 0 22.0 0.0 -22.0 1750.166 0.0 -22.0 0.0 -10.175 22.0 -1.1945454545454546 22.0 0.0 22 -NULL 45 1969-12-31 16:00:00.304 45.0 NULL -45 45 0 45.0 0.0 -45.0 3579.8849999999998 0.0 -45.0 0.0 -10.175 45.0 -0.5840000000000001 45.0 0.0 45 -true -25 1969-12-31 16:00:13.978 -25.0 821UdmGbkEf4j 25 -25 0 -25.0 -0.0 25.0 -1988.8249999999998 0.0 25.0 0.0 -10.175 -25.0 1.0512000000000001 -25.0 0.0 -25 -NULL -5 1969-12-31 16:00:14.505 -5.0 NULL 5 -5 0 -5.0 -0.0 5.0 -397.765 0.0 5.0 0.0 -10.175 -5.0 5.256 -5.0 0.0 -5 -true -64 1969-12-31 16:00:11.952 -64.0 821UdmGbkEf4j 64 -64 0 -64.0 -0.0 64.0 -5091.392 0.0 64.0 0.0 -10.175 -64.0 0.410625 -64.0 0.0 -64 -true 60 1969-12-31 16:00:02.958 60.0 1cGVWH7n1QU -60 60 0 60.0 0.0 -60.0 4773.18 0.0 -60.0 0.0 -10.175 60.0 -0.438 60.0 0.0 60 -true -28 1969-12-31 16:00:12.375 -28.0 821UdmGbkEf4j 28 -28 0 -28.0 -0.0 28.0 -2227.484 0.0 28.0 0.0 -10.175 -28.0 0.9385714285714286 -28.0 0.0 -28 -true 57 1969-12-31 16:00:01.967 57.0 cvLH6Eat2yFsyy7p -57 57 0 57.0 0.0 -57.0 4534.521 0.0 -57.0 0.0 -10.175 57.0 -0.4610526315789474 57.0 0.0 57 -true 21 1969-12-31 16:00:11.525 21.0 1cGVWH7n1QU -21 21 0 21.0 0.0 -21.0 1670.6129999999998 0.0 -21.0 0.0 -10.175 21.0 -1.2514285714285716 21.0 0.0 21 -true 37 1969-12-31 16:00:12.643 37.0 821UdmGbkEf4j -37 37 0 37.0 0.0 -37.0 2943.461 0.0 -37.0 0.0 -10.175 37.0 -0.7102702702702703 37.0 0.0 37 -NULL -14 1969-12-31 16:00:07.785 -14.0 NULL 14 -14 0 -14.0 -0.0 14.0 -1113.742 0.0 14.0 0.0 -10.175 -14.0 1.8771428571428572 -14.0 0.0 -14 -true 50 1969-12-31 16:00:13.169 50.0 cvLH6Eat2yFsyy7p -50 50 0 50.0 0.0 -50.0 3977.6499999999996 0.0 -50.0 0.0 -10.175 50.0 -0.5256000000000001 50.0 0.0 50 -true 61 1969-12-31 16:00:15.101 61.0 821UdmGbkEf4j -61 61 0 61.0 0.0 -61.0 4852.733 0.0 -61.0 0.0 -10.175 61.0 -0.4308196721311476 61.0 0.0 61 -true 3 1969-12-31 16:00:13.736 3.0 1cGVWH7n1QU -3 3 0 3.0 0.0 -3.0 238.659 0.0 -3.0 0.0 -10.175 3.0 -8.76 3.0 0.0 3 -NULL 62 1969-12-31 16:00:11.213 62.0 NULL -62 62 0 62.0 0.0 -62.0 4932.286 0.0 -62.0 0.0 -10.175 62.0 -0.4238709677419355 62.0 0.0 62 -NULL -47 1969-12-31 16:00:00.235 -47.0 NULL 47 -47 0 -47.0 -0.0 47.0 -3738.991 0.0 47.0 0.0 -10.175 -47.0 0.5591489361702128 -47.0 0.0 -47 -NULL 50 1969-12-31 16:00:11.048 50.0 NULL -50 50 0 50.0 0.0 -50.0 3977.6499999999996 0.0 -50.0 0.0 -10.175 50.0 -0.5256000000000001 50.0 0.0 50 -NULL -42 1969-12-31 16:00:05.729 -42.0 NULL 42 -42 0 -42.0 -0.0 42.0 -3341.2259999999997 0.0 42.0 0.0 -10.175 -42.0 0.6257142857142858 -42.0 0.0 -42 -NULL 33 1969-12-31 16:00:04.568 79.553 NULL -33 33 0 79.5530014038086 0.0 -79.5530014038086 6328.679920677185 0.0 -79.5530014038086 0.0 -10.175 79.5530014038086 -0.33034580136836733 79.553 0.0 33 -NULL -33 1969-12-31 16:00:15.375 -33.0 NULL 33 -33 0 -33.0 -0.0 33.0 -2625.249 0.0 33.0 0.0 -10.175 -33.0 0.7963636363636364 -33.0 0.0 -33 -NULL 60 1969-12-31 16:00:13.71 60.0 NULL -60 60 0 60.0 0.0 -60.0 4773.18 0.0 -60.0 0.0 -10.175 60.0 -0.438 60.0 0.0 60 -true -6 1969-12-31 16:00:11.794 -6.0 cvLH6Eat2yFsyy7p 6 -6 0 -6.0 -0.0 6.0 -477.318 0.0 6.0 0.0 -10.175 -6.0 4.38 -6.0 0.0 -6 -true -61 1969-12-31 16:00:09.809 -61.0 1cGVWH7n1QU 61 -61 0 -61.0 -0.0 61.0 -4852.733 0.0 61.0 0.0 -10.175 -61.0 0.4308196721311476 -61.0 0.0 -61 -true 14 1969-12-31 16:00:01.815 14.0 cvLH6Eat2yFsyy7p -14 14 0 14.0 0.0 -14.0 1113.742 0.0 -14.0 0.0 -10.175 14.0 -1.8771428571428572 14.0 0.0 14 -true 3 1969-12-31 16:00:11.471 3.0 1cGVWH7n1QU -3 3 0 3.0 0.0 -3.0 238.659 0.0 -3.0 0.0 -10.175 3.0 -8.76 3.0 0.0 3 -NULL 50 1969-12-31 16:00:04.228 50.0 NULL -50 50 0 50.0 0.0 -50.0 3977.6499999999996 0.0 -50.0 0.0 -10.175 50.0 -0.5256000000000001 50.0 0.0 50 -true -34 1969-12-31 16:00:10.696 -34.0 1cGVWH7n1QU 34 -34 0 -34.0 -0.0 34.0 -2704.8019999999997 0.0 34.0 0.0 -10.175 -34.0 0.7729411764705882 -34.0 0.0 -34 -NULL -19 1969-12-31 16:00:05.972 -19.0 NULL 19 -19 0 -19.0 -0.0 19.0 -1511.507 0.0 19.0 0.0 -10.175 -19.0 1.3831578947368421 -19.0 0.0 -19 -NULL 5 1969-12-31 16:00:00.959 5.0 NULL -5 5 0 5.0 0.0 -5.0 397.765 0.0 -5.0 0.0 -10.175 5.0 -5.256 5.0 0.0 5 -NULL -61 1969-12-31 16:00:04.165 -61.0 NULL 61 -61 0 -61.0 -0.0 61.0 -4852.733 0.0 61.0 0.0 -10.175 -61.0 0.4308196721311476 -61.0 0.0 -61 -NULL 10 1969-12-31 16:00:03.139 10.0 NULL -10 10 0 10.0 0.0 -10.0 795.53 0.0 -10.0 0.0 -10.175 10.0 -2.628 10.0 0.0 10 -NULL -9 1969-12-31 16:00:11.758 -9.0 NULL 9 -9 0 -9.0 -0.0 9.0 -715.977 0.0 9.0 0.0 -10.175 -9.0 2.92 -9.0 0.0 -9 -true 34 1969-12-31 16:00:00.535 34.0 1cGVWH7n1QU -34 34 0 34.0 0.0 -34.0 2704.8019999999997 0.0 -34.0 0.0 -10.175 34.0 -0.7729411764705882 34.0 0.0 34 -NULL -37 1969-12-31 16:00:08.141 -37.0 NULL 37 -37 0 -37.0 -0.0 37.0 -2943.461 0.0 37.0 0.0 -10.175 -37.0 0.7102702702702703 -37.0 0.0 -37 -true -24 1969-12-31 16:00:10.461 -24.0 cvLH6Eat2yFsyy7p 24 -24 0 -24.0 -0.0 24.0 -1909.272 0.0 24.0 0.0 -10.175 -24.0 1.095 -24.0 0.0 -24 -NULL -38 1969-12-31 16:00:07.111 -38.0 NULL 38 -38 0 -38.0 -0.0 38.0 -3023.014 0.0 38.0 0.0 -10.175 -38.0 0.6915789473684211 -38.0 0.0 -38 -true -35 1969-12-31 16:00:09.395 -35.0 821UdmGbkEf4j 35 -35 0 -35.0 -0.0 35.0 -2784.355 0.0 35.0 0.0 -10.175 -35.0 0.7508571428571429 -35.0 0.0 -35 -NULL -12 1969-12-31 16:00:13.678 -12.0 NULL 12 -12 0 -12.0 -0.0 12.0 -954.636 0.0 12.0 0.0 -10.175 -12.0 2.19 -12.0 0.0 -12 -true 5 1969-12-31 16:00:05.015 5.0 cvLH6Eat2yFsyy7p -5 5 0 5.0 0.0 -5.0 397.765 0.0 -5.0 0.0 -10.175 5.0 -5.256 5.0 0.0 5 -NULL -41 1969-12-31 16:00:15.69 -41.0 NULL 41 -41 0 -41.0 -0.0 41.0 -3261.673 0.0 41.0 0.0 -10.175 -41.0 0.6409756097560976 -41.0 0.0 -41 -NULL -16 1969-12-31 16:00:10.394 -16.0 NULL 16 -16 0 -16.0 -0.0 16.0 -1272.848 0.0 16.0 0.0 -10.175 -16.0 1.6425 -16.0 0.0 -16 -true 49 1969-12-31 16:00:11.401 49.0 821UdmGbkEf4j -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 -NULL 61 1969-12-31 16:00:01.505 61.0 NULL -61 61 0 61.0 0.0 -61.0 4852.733 0.0 -61.0 0.0 -10.175 61.0 -0.4308196721311476 61.0 0.0 61 -true 47 1969-12-31 16:00:12.778 47.0 cvLH6Eat2yFsyy7p -47 47 0 47.0 0.0 -47.0 3738.991 0.0 -47.0 0.0 -10.175 47.0 -0.5591489361702128 47.0 0.0 47 -NULL -15 1969-12-31 16:00:05.113 -15.0 NULL 15 -15 0 -15.0 -0.0 15.0 -1193.295 0.0 15.0 0.0 -10.175 -15.0 1.752 -15.0 0.0 -15 -true -20 1969-12-31 16:00:10.485 -20.0 cvLH6Eat2yFsyy7p 20 -20 0 -20.0 -0.0 20.0 -1591.06 0.0 20.0 0.0 -10.175 -20.0 1.314 -20.0 0.0 -20 -NULL 4 1969-12-31 16:00:11.457 4.0 NULL -4 4 0 4.0 0.0 -4.0 318.212 0.0 -4.0 0.0 -10.175 4.0 -6.57 4.0 0.0 4 -true 11 1969-12-31 16:00:13.613 11.0 cvLH6Eat2yFsyy7p -11 11 0 11.0 0.0 -11.0 875.083 0.0 -11.0 0.0 -10.175 11.0 -2.389090909090909 11.0 0.0 11 -NULL -57 1969-12-31 16:00:11.451 -57.0 NULL 57 -57 0 -57.0 -0.0 57.0 -4534.521 0.0 57.0 0.0 -10.175 -57.0 0.4610526315789474 -57.0 0.0 -57 -NULL 19 1969-12-31 16:00:13.074 19.0 NULL -19 19 0 19.0 0.0 -19.0 1511.507 0.0 -19.0 0.0 -10.175 19.0 -1.3831578947368421 19.0 0.0 19 -NULL -12 1969-12-31 16:00:15.92 -12.0 NULL 12 -12 0 -12.0 -0.0 12.0 -954.636 0.0 12.0 0.0 -10.175 -12.0 2.19 -12.0 0.0 -12 -NULL -9 1969-12-31 16:00:13.473 -9.0 NULL 9 -9 0 -9.0 -0.0 9.0 -715.977 0.0 9.0 0.0 -10.175 -9.0 2.92 -9.0 0.0 -9 -NULL 48 1969-12-31 16:00:16.19 48.0 NULL -48 48 0 48.0 0.0 -48.0 3818.544 0.0 -48.0 0.0 -10.175 48.0 -0.5475 48.0 0.0 48 -true -60 1969-12-31 16:00:06.658 -60.0 821UdmGbkEf4j 60 -60 0 -60.0 -0.0 60.0 -4773.18 0.0 60.0 0.0 -10.175 -60.0 0.438 -60.0 0.0 -60 -true -51 1969-12-31 16:00:13.085 -51.0 1cGVWH7n1QU 51 -51 0 -51.0 -0.0 51.0 -4057.203 0.0 51.0 0.0 -10.175 -51.0 0.5152941176470588 -51.0 0.0 -51 -true -43 1969-12-31 16:00:04.539 -43.0 821UdmGbkEf4j 43 -43 0 -43.0 -0.0 43.0 -3420.779 0.0 43.0 0.0 -10.175 -43.0 0.6111627906976744 -43.0 0.0 -43 -NULL -38 1969-12-31 16:00:09.644 -38.0 NULL 38 -38 0 -38.0 -0.0 38.0 -3023.014 0.0 38.0 0.0 -10.175 -38.0 0.6915789473684211 -38.0 0.0 -38 -true -41 1969-12-31 16:00:02.99 -41.0 1cGVWH7n1QU 41 -41 0 -41.0 -0.0 41.0 -3261.673 0.0 41.0 0.0 -10.175 -41.0 0.6409756097560976 -41.0 0.0 -41 -true 59 1969-12-31 16:00:07.546 59.0 1cGVWH7n1QU -59 59 0 59.0 0.0 -59.0 4693.6269999999995 0.0 -59.0 0.0 -10.175 59.0 -0.44542372881355935 59.0 0.0 59 -true 48 1969-12-31 16:00:04.561 48.0 1cGVWH7n1QU -48 48 0 48.0 0.0 -48.0 3818.544 0.0 -48.0 0.0 -10.175 48.0 -0.5475 48.0 0.0 48 -NULL 45 1969-12-31 16:00:16.149 45.0 NULL -45 45 0 45.0 0.0 -45.0 3579.8849999999998 0.0 -45.0 0.0 -10.175 45.0 -0.5840000000000001 45.0 0.0 45 -NULL 58 1969-12-31 16:00:11.449 58.0 NULL -58 58 0 58.0 0.0 -58.0 4614.074 0.0 -58.0 0.0 -10.175 58.0 -0.4531034482758621 58.0 0.0 58 -NULL 47 1969-12-31 16:00:12.183 47.0 NULL -47 47 0 47.0 0.0 -47.0 3738.991 0.0 -47.0 0.0 -10.175 47.0 -0.5591489361702128 47.0 0.0 47 -true 30 1969-12-31 16:00:09.227 30.0 821UdmGbkEf4j -30 30 0 30.0 0.0 -30.0 2386.59 0.0 -30.0 0.0 -10.175 30.0 -0.876 30.0 0.0 30 -NULL -55 1969-12-31 16:00:08.229 -55.0 NULL 55 -55 0 -55.0 -0.0 55.0 -4375.415 0.0 55.0 0.0 -10.175 -55.0 0.47781818181818186 -55.0 0.0 -55 -true 42 1969-12-31 16:00:14.152 42.0 821UdmGbkEf4j -42 42 0 42.0 0.0 -42.0 3341.2259999999997 0.0 -42.0 0.0 -10.175 42.0 -0.6257142857142858 42.0 0.0 42 -NULL -15 1969-12-31 16:00:11.257 -15.0 NULL 15 -15 0 -15.0 -0.0 15.0 -1193.295 0.0 15.0 0.0 -10.175 -15.0 1.752 -15.0 0.0 -15 -true -54 1969-12-31 16:00:01.931 -54.0 cvLH6Eat2yFsyy7p 54 -54 0 -54.0 -0.0 54.0 -4295.862 0.0 54.0 0.0 -10.175 -54.0 0.4866666666666667 -54.0 0.0 -54 -NULL -7 1969-12-31 16:00:04.199 -7.0 NULL 7 -7 0 -7.0 -0.0 7.0 -556.871 0.0 7.0 0.0 -10.175 -7.0 3.7542857142857144 -7.0 0.0 -7 -true 3 1969-12-31 16:00:03.256 3.0 821UdmGbkEf4j -3 3 0 3.0 0.0 -3.0 238.659 0.0 -3.0 0.0 -10.175 3.0 -8.76 3.0 0.0 3 -NULL -44 1969-12-31 16:00:03.265 -44.0 NULL 44 -44 0 -44.0 -0.0 44.0 -3500.332 0.0 44.0 0.0 -10.175 -44.0 0.5972727272727273 -44.0 0.0 -44 -true 37 1969-12-31 16:00:05.645 37.0 821UdmGbkEf4j -37 37 0 37.0 0.0 -37.0 2943.461 0.0 -37.0 0.0 -10.175 37.0 -0.7102702702702703 37.0 0.0 37 -NULL -58 1969-12-31 16:00:09.622 -58.0 NULL 58 -58 0 -58.0 -0.0 58.0 -4614.074 0.0 58.0 0.0 -10.175 -58.0 0.4531034482758621 -58.0 0.0 -58 -NULL -19 1969-12-31 16:00:01.564 -19.0 NULL 19 -19 0 -19.0 -0.0 19.0 -1511.507 0.0 19.0 0.0 -10.175 -19.0 1.3831578947368421 -19.0 0.0 -19 -true 42 1969-12-31 16:00:00.865 42.0 1cGVWH7n1QU -42 42 0 42.0 0.0 -42.0 3341.2259999999997 0.0 -42.0 0.0 -10.175 42.0 -0.6257142857142858 42.0 0.0 42 -true 37 1969-12-31 16:00:08.438 37.0 1cGVWH7n1QU -37 37 0 37.0 0.0 -37.0 2943.461 0.0 -37.0 0.0 -10.175 37.0 -0.7102702702702703 37.0 0.0 37 -true -44 1969-12-31 16:00:08.237 -44.0 821UdmGbkEf4j 44 -44 0 -44.0 -0.0 44.0 -3500.332 0.0 44.0 0.0 -10.175 -44.0 0.5972727272727273 -44.0 0.0 -44 -NULL 14 1969-12-31 16:00:00.931 14.0 NULL -14 14 0 14.0 0.0 -14.0 1113.742 0.0 -14.0 0.0 -10.175 14.0 -1.8771428571428572 14.0 0.0 14 -NULL 4 1969-12-31 16:00:07.163 4.0 NULL -4 4 0 4.0 0.0 -4.0 318.212 0.0 -4.0 0.0 -10.175 4.0 -6.57 4.0 0.0 4 -true -23 1969-12-31 16:00:00.859 -23.0 cvLH6Eat2yFsyy7p 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -NULL 0 1969-12-31 16:00:15.629 0.0 NULL 0 0 0 0.0 0.0 -0.0 0.0 0.0 -0.0 0.0 -10.175 0.0 NULL 0.0 NULL 0 -true -35 1969-12-31 16:00:16.174 -35.0 821UdmGbkEf4j 35 -35 0 -35.0 -0.0 35.0 -2784.355 0.0 35.0 0.0 -10.175 -35.0 0.7508571428571429 -35.0 0.0 -35 -NULL 61 1969-12-31 16:00:01.11 61.0 NULL -61 61 0 61.0 0.0 -61.0 4852.733 0.0 -61.0 0.0 -10.175 61.0 -0.4308196721311476 61.0 0.0 61 -true -58 1969-12-31 16:00:03.684 -58.0 821UdmGbkEf4j 58 -58 0 -58.0 -0.0 58.0 -4614.074 0.0 58.0 0.0 -10.175 -58.0 0.4531034482758621 -58.0 0.0 -58 -true 53 1969-12-31 16:00:08.79 53.0 cvLH6Eat2yFsyy7p -53 53 0 53.0 0.0 -53.0 4216.309 0.0 -53.0 0.0 -10.175 53.0 -0.4958490566037736 53.0 0.0 53 -true -29 1969-12-31 16:00:14.504 -29.0 1cGVWH7n1QU 29 -29 0 -29.0 -0.0 29.0 -2307.037 0.0 29.0 0.0 -10.175 -29.0 0.9062068965517242 -29.0 0.0 -29 -true 48 1969-12-31 16:00:00.511 48.0 821UdmGbkEf4j -48 48 0 48.0 0.0 -48.0 3818.544 0.0 -48.0 0.0 -10.175 48.0 -0.5475 48.0 0.0 48 -NULL -27 1969-12-31 16:00:06.385 -27.0 NULL 27 -27 0 -27.0 -0.0 27.0 -2147.931 0.0 27.0 0.0 -10.175 -27.0 0.9733333333333334 -27.0 0.0 -27 -true -55 1969-12-31 16:00:06.667 -55.0 cvLH6Eat2yFsyy7p 55 -55 0 -55.0 -0.0 55.0 -4375.415 0.0 55.0 0.0 -10.175 -55.0 0.47781818181818186 -55.0 0.0 -55 -true -39 1969-12-31 16:00:16.219 -39.0 821UdmGbkEf4j 39 -39 0 -39.0 -0.0 39.0 -3102.567 0.0 39.0 0.0 -10.175 -39.0 0.6738461538461539 -39.0 0.0 -39 -NULL -49 1969-12-31 16:00:04.168 -49.0 NULL 49 -49 0 -49.0 -0.0 49.0 -3898.0969999999998 0.0 49.0 0.0 -10.175 -49.0 0.5363265306122449 -49.0 0.0 -49 -true 35 1969-12-31 16:00:15.668 35.0 cvLH6Eat2yFsyy7p -35 35 0 35.0 0.0 -35.0 2784.355 0.0 -35.0 0.0 -10.175 35.0 -0.7508571428571429 35.0 0.0 35 -true -31 1969-12-31 16:00:14.694 -31.0 cvLH6Eat2yFsyy7p 31 -31 0 -31.0 -0.0 31.0 -2466.143 0.0 31.0 0.0 -10.175 -31.0 0.847741935483871 -31.0 0.0 -31 -true 43 1969-12-31 16:00:15.933 43.0 cvLH6Eat2yFsyy7p -43 43 0 43.0 0.0 -43.0 3420.779 0.0 -43.0 0.0 -10.175 43.0 -0.6111627906976744 43.0 0.0 43 -true 14 1969-12-31 16:00:00.546 14.0 821UdmGbkEf4j -14 14 0 14.0 0.0 -14.0 1113.742 0.0 -14.0 0.0 -10.175 14.0 -1.8771428571428572 14.0 0.0 14 -true -24 1969-12-31 16:00:06.321 -24.0 821UdmGbkEf4j 24 -24 0 -24.0 -0.0 24.0 -1909.272 0.0 24.0 0.0 -10.175 -24.0 1.095 -24.0 0.0 -24 -NULL -2 1969-12-31 16:00:07.711 -2.0 NULL 2 -2 0 -2.0 -0.0 2.0 -159.106 0.0 2.0 0.0 -10.175 -2.0 13.14 -2.0 0.0 -2 -true 4 1969-12-31 16:00:05.752 4.0 821UdmGbkEf4j -4 4 0 4.0 0.0 -4.0 318.212 0.0 -4.0 0.0 -10.175 4.0 -6.57 4.0 0.0 4 -NULL -44 1969-12-31 16:00:00.289 -44.0 NULL 44 -44 0 -44.0 -0.0 44.0 -3500.332 0.0 44.0 0.0 -10.175 -44.0 0.5972727272727273 -44.0 0.0 -44 -true -57 1969-12-31 16:00:08.709 -57.0 821UdmGbkEf4j 57 -57 0 -57.0 -0.0 57.0 -4534.521 0.0 57.0 0.0 -10.175 -57.0 0.4610526315789474 -57.0 0.0 -57 -true -23 1969-12-31 16:00:12.734 -23.0 821UdmGbkEf4j 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -NULL 36 1969-12-31 16:00:00.554 36.0 NULL -36 36 0 36.0 0.0 -36.0 2863.908 0.0 -36.0 0.0 -10.175 36.0 -0.73 36.0 0.0 36 -NULL -3 1969-12-31 16:00:03.349 -3.0 NULL 3 -3 0 -3.0 -0.0 3.0 -238.659 0.0 3.0 0.0 -10.175 -3.0 8.76 -3.0 0.0 -3 -true 17 1969-12-31 16:00:08.246 17.0 1cGVWH7n1QU -17 17 0 17.0 0.0 -17.0 1352.4009999999998 0.0 -17.0 0.0 -10.175 17.0 -1.5458823529411765 17.0 0.0 17 -NULL 54 1969-12-31 16:00:14.808 54.0 NULL -54 54 0 54.0 0.0 -54.0 4295.862 0.0 -54.0 0.0 -10.175 54.0 -0.4866666666666667 54.0 0.0 54 -NULL 17 1969-12-31 16:00:07.548 17.0 NULL -17 17 0 17.0 0.0 -17.0 1352.4009999999998 0.0 -17.0 0.0 -10.175 17.0 -1.5458823529411765 17.0 0.0 17 -true -26 1969-12-31 16:00:10.915 -26.0 1cGVWH7n1QU 26 -26 0 -26.0 -0.0 26.0 -2068.3779999999997 0.0 26.0 0.0 -10.175 -26.0 1.0107692307692309 -26.0 0.0 -26 -true 15 1969-12-31 16:00:06.748 15.0 1cGVWH7n1QU -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -true 20 1969-12-31 16:00:07.813 20.0 821UdmGbkEf4j -20 20 0 20.0 0.0 -20.0 1591.06 0.0 -20.0 0.0 -10.175 20.0 -1.314 20.0 0.0 20 -true -56 1969-12-31 16:00:05.851 -56.0 cvLH6Eat2yFsyy7p 56 -56 0 -56.0 -0.0 56.0 -4454.968 0.0 56.0 0.0 -10.175 -56.0 0.4692857142857143 -56.0 0.0 -56 -true -51 1969-12-31 16:00:11.394 -51.0 cvLH6Eat2yFsyy7p 51 -51 0 -51.0 -0.0 51.0 -4057.203 0.0 51.0 0.0 -10.175 -51.0 0.5152941176470588 -51.0 0.0 -51 -NULL -20 1969-12-31 16:00:13.131 -20.0 NULL 20 -20 0 -20.0 -0.0 20.0 -1591.06 0.0 20.0 0.0 -10.175 -20.0 1.314 -20.0 0.0 -20 -true 30 1969-12-31 16:00:15.705 30.0 821UdmGbkEf4j -30 30 0 30.0 0.0 -30.0 2386.59 0.0 -30.0 0.0 -10.175 30.0 -0.876 30.0 0.0 30 -true 45 1969-12-31 16:00:15.985 45.0 cvLH6Eat2yFsyy7p -45 45 0 45.0 0.0 -45.0 3579.8849999999998 0.0 -45.0 0.0 -10.175 45.0 -0.5840000000000001 45.0 0.0 45 -true -57 1969-12-31 16:00:11.534 -57.0 cvLH6Eat2yFsyy7p 57 -57 0 -57.0 -0.0 57.0 -4534.521 0.0 57.0 0.0 -10.175 -57.0 0.4610526315789474 -57.0 0.0 -57 -NULL -37 1969-12-31 16:00:15.215 -37.0 NULL 37 -37 0 -37.0 -0.0 37.0 -2943.461 0.0 37.0 0.0 -10.175 -37.0 0.7102702702702703 -37.0 0.0 -37 -NULL -43 1969-12-31 16:00:08.967 -43.0 NULL 43 -43 0 -43.0 -0.0 43.0 -3420.779 0.0 43.0 0.0 -10.175 -43.0 0.6111627906976744 -43.0 0.0 -43 -NULL 2 1969-12-31 16:00:16.324 2.0 NULL -2 2 0 2.0 0.0 -2.0 159.106 0.0 -2.0 0.0 -10.175 2.0 -13.14 2.0 0.0 2 -NULL 55 1969-12-31 16:00:02.488 55.0 NULL -55 55 0 55.0 0.0 -55.0 4375.415 0.0 -55.0 0.0 -10.175 55.0 -0.47781818181818186 55.0 0.0 55 -NULL -45 1969-12-31 16:00:07.391 -45.0 NULL 45 -45 0 -45.0 -0.0 45.0 -3579.8849999999998 0.0 45.0 0.0 -10.175 -45.0 0.5840000000000001 -45.0 0.0 -45 -true -30 1969-12-31 16:00:07.185 -30.0 1cGVWH7n1QU 30 -30 0 -30.0 -0.0 30.0 -2386.59 0.0 30.0 0.0 -10.175 -30.0 0.876 -30.0 0.0 -30 -true -61 1969-12-31 16:00:00.554 -61.0 1cGVWH7n1QU 61 -61 0 -61.0 -0.0 61.0 -4852.733 0.0 61.0 0.0 -10.175 -61.0 0.4308196721311476 -61.0 0.0 -61 -NULL -46 1969-12-31 16:00:08.334 -46.0 NULL 46 -46 0 -46.0 -0.0 46.0 -3659.438 0.0 46.0 0.0 -10.175 -46.0 0.571304347826087 -46.0 0.0 -46 -NULL -23 1969-12-31 16:00:00.885 -23.0 NULL 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -true 27 1969-12-31 16:00:02.384 27.0 cvLH6Eat2yFsyy7p -27 27 0 27.0 0.0 -27.0 2147.931 0.0 -27.0 0.0 -10.175 27.0 -0.9733333333333334 27.0 0.0 27 -NULL 0 1969-12-31 16:00:15.777 0.0 NULL 0 0 0 0.0 0.0 -0.0 0.0 0.0 -0.0 0.0 -10.175 0.0 NULL 0.0 NULL 0 -NULL 62 1969-12-31 16:00:02.861 62.0 NULL -62 62 0 62.0 0.0 -62.0 4932.286 0.0 -62.0 0.0 -10.175 62.0 -0.4238709677419355 62.0 0.0 62 -NULL -39 1969-12-31 16:00:00.39 -39.0 NULL 39 -39 0 -39.0 -0.0 39.0 -3102.567 0.0 39.0 0.0 -10.175 -39.0 0.6738461538461539 -39.0 0.0 -39 -NULL -56 1969-12-31 16:00:00.417 -56.0 NULL 56 -56 0 -56.0 -0.0 56.0 -4454.968 0.0 56.0 0.0 -10.175 -56.0 0.4692857142857143 -56.0 0.0 -56 -true -6 1969-12-31 16:00:06.144 -6.0 1cGVWH7n1QU 6 -6 0 -6.0 -0.0 6.0 -477.318 0.0 6.0 0.0 -10.175 -6.0 4.38 -6.0 0.0 -6 -NULL -62 1969-12-31 16:00:09.889 -62.0 NULL 62 -62 0 -62.0 -0.0 62.0 -4932.286 0.0 62.0 0.0 -10.175 -62.0 0.4238709677419355 -62.0 0.0 -62 -NULL -22 1969-12-31 16:00:07.969 -22.0 NULL 22 -22 0 -22.0 -0.0 22.0 -1750.166 0.0 22.0 0.0 -10.175 -22.0 1.1945454545454546 -22.0 0.0 -22 -NULL -45 1969-12-31 16:00:12.921 -45.0 NULL 45 -45 0 -45.0 -0.0 45.0 -3579.8849999999998 0.0 45.0 0.0 -10.175 -45.0 0.5840000000000001 -45.0 0.0 -45 -true 60 1969-12-31 16:00:08.462 60.0 1cGVWH7n1QU -60 60 0 60.0 0.0 -60.0 4773.18 0.0 -60.0 0.0 -10.175 60.0 -0.438 60.0 0.0 60 -NULL 9 1969-12-31 16:00:09.285 9.0 NULL -9 9 0 9.0 0.0 -9.0 715.977 0.0 -9.0 0.0 -10.175 9.0 -2.92 9.0 0.0 9 -NULL -13 1969-12-31 16:00:09.357 -13.0 NULL 13 -13 0 -13.0 -0.0 13.0 -1034.1889999999999 0.0 13.0 0.0 -10.175 -13.0 2.0215384615384617 -13.0 0.0 -13 -NULL -24 1969-12-31 16:00:07.857 -24.0 NULL 24 -24 0 -24.0 -0.0 24.0 -1909.272 0.0 24.0 0.0 -10.175 -24.0 1.095 -24.0 0.0 -24 -NULL -34 1969-12-31 16:00:09.894 -34.0 NULL 34 -34 0 -34.0 -0.0 34.0 -2704.8019999999997 0.0 34.0 0.0 -10.175 -34.0 0.7729411764705882 -34.0 0.0 -34 -NULL -45 1969-12-31 16:00:12.648 -45.0 NULL 45 -45 0 -45.0 -0.0 45.0 -3579.8849999999998 0.0 45.0 0.0 -10.175 -45.0 0.5840000000000001 -45.0 0.0 -45 -true 41 1969-12-31 16:00:00.832 41.0 cvLH6Eat2yFsyy7p -41 41 0 41.0 0.0 -41.0 3261.673 0.0 -41.0 0.0 -10.175 41.0 -0.6409756097560976 41.0 0.0 41 -NULL -31 1969-12-31 16:00:00.882 -31.0 NULL 31 -31 0 -31.0 -0.0 31.0 -2466.143 0.0 31.0 0.0 -10.175 -31.0 0.847741935483871 -31.0 0.0 -31 -NULL -62 1969-12-31 16:00:02.373 -62.0 NULL 62 -62 0 -62.0 -0.0 62.0 -4932.286 0.0 62.0 0.0 -10.175 -62.0 0.4238709677419355 -62.0 0.0 -62 -NULL -36 1969-12-31 16:00:15.277 -36.0 NULL 36 -36 0 -36.0 -0.0 36.0 -2863.908 0.0 36.0 0.0 -10.175 -36.0 0.73 -36.0 0.0 -36 -NULL 20 1969-12-31 16:00:09.778 20.0 NULL -20 20 0 20.0 0.0 -20.0 1591.06 0.0 -20.0 0.0 -10.175 20.0 -1.314 20.0 0.0 20 -true -46 1969-12-31 16:00:05.815 -46.0 821UdmGbkEf4j 46 -46 0 -46.0 -0.0 46.0 -3659.438 0.0 46.0 0.0 -10.175 -46.0 0.571304347826087 -46.0 0.0 -46 -NULL 48 1969-12-31 16:00:00.008 48.0 NULL -48 48 0 48.0 0.0 -48.0 3818.544 0.0 -48.0 0.0 -10.175 48.0 -0.5475 48.0 0.0 48 -true 45 1969-12-31 16:00:05.308 45.0 1cGVWH7n1QU -45 45 0 45.0 0.0 -45.0 3579.8849999999998 0.0 -45.0 0.0 -10.175 45.0 -0.5840000000000001 45.0 0.0 45 -true 34 1969-12-31 16:00:05.177 34.0 1cGVWH7n1QU -34 34 0 34.0 0.0 -34.0 2704.8019999999997 0.0 -34.0 0.0 -10.175 34.0 -0.7729411764705882 34.0 0.0 34 -true -60 1969-12-31 16:00:02.636 -60.0 1cGVWH7n1QU 60 -60 0 -60.0 -0.0 60.0 -4773.18 0.0 60.0 0.0 -10.175 -60.0 0.438 -60.0 0.0 -60 -true -24 1969-12-31 16:00:11.23 -24.0 cvLH6Eat2yFsyy7p 24 -24 0 -24.0 -0.0 24.0 -1909.272 0.0 24.0 0.0 -10.175 -24.0 1.095 -24.0 0.0 -24 -true 9 1969-12-31 16:00:09.063 9.0 821UdmGbkEf4j -9 9 0 9.0 0.0 -9.0 715.977 0.0 -9.0 0.0 -10.175 9.0 -2.92 9.0 0.0 9 -true -18 1969-12-31 16:00:12.23 -18.0 cvLH6Eat2yFsyy7p 18 -18 0 -18.0 -0.0 18.0 -1431.954 0.0 18.0 0.0 -10.175 -18.0 1.46 -18.0 0.0 -18 -NULL -44 1969-12-31 16:00:15.014 -44.0 NULL 44 -44 0 -44.0 -0.0 44.0 -3500.332 0.0 44.0 0.0 -10.175 -44.0 0.5972727272727273 -44.0 0.0 -44 -true 39 1969-12-31 16:00:00.494 39.0 cvLH6Eat2yFsyy7p -39 39 0 39.0 0.0 -39.0 3102.567 0.0 -39.0 0.0 -10.175 39.0 -0.6738461538461539 39.0 0.0 39 -NULL -15 1969-12-31 16:00:13.058 -15.0 NULL 15 -15 0 -15.0 -0.0 15.0 -1193.295 0.0 15.0 0.0 -10.175 -15.0 1.752 -15.0 0.0 -15 -true 0 1969-12-31 16:00:02.284 0.0 cvLH6Eat2yFsyy7p 0 0 0 0.0 0.0 -0.0 0.0 0.0 -0.0 0.0 -10.175 0.0 NULL 0.0 NULL 0 -true 17 1969-12-31 16:00:11.246 17.0 821UdmGbkEf4j -17 17 0 17.0 0.0 -17.0 1352.4009999999998 0.0 -17.0 0.0 -10.175 17.0 -1.5458823529411765 17.0 0.0 17 -NULL -27 1969-12-31 16:00:00.692 -27.0 NULL 27 -27 0 -27.0 -0.0 27.0 -2147.931 0.0 27.0 0.0 -10.175 -27.0 0.9733333333333334 -27.0 0.0 -27 -NULL -38 1969-12-31 16:00:12.213 -38.0 NULL 38 -38 0 -38.0 -0.0 38.0 -3023.014 0.0 38.0 0.0 -10.175 -38.0 0.6915789473684211 -38.0 0.0 -38 -true 48 1969-12-31 16:00:00.905 48.0 821UdmGbkEf4j -48 48 0 48.0 0.0 -48.0 3818.544 0.0 -48.0 0.0 -10.175 48.0 -0.5475 48.0 0.0 48 -true -56 1969-12-31 16:00:05.674 -56.0 1cGVWH7n1QU 56 -56 0 -56.0 -0.0 56.0 -4454.968 0.0 56.0 0.0 -10.175 -56.0 0.4692857142857143 -56.0 0.0 -56 -true 24 1969-12-31 16:00:07.86 24.0 cvLH6Eat2yFsyy7p -24 24 0 24.0 0.0 -24.0 1909.272 0.0 -24.0 0.0 -10.175 24.0 -1.095 24.0 0.0 24 -true -30 1969-12-31 16:00:14.53 -30.0 821UdmGbkEf4j 30 -30 0 -30.0 -0.0 30.0 -2386.59 0.0 30.0 0.0 -10.175 -30.0 0.876 -30.0 0.0 -30 -NULL -23 1969-12-31 16:00:07.71 -23.0 NULL 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -NULL 4 1969-12-31 16:00:07.836 4.0 NULL -4 4 0 4.0 0.0 -4.0 318.212 0.0 -4.0 0.0 -10.175 4.0 -6.57 4.0 0.0 4 -NULL -48 1969-12-31 16:00:14.704 -48.0 NULL 48 -48 0 -48.0 -0.0 48.0 -3818.544 0.0 48.0 0.0 -10.175 -48.0 0.5475 -48.0 0.0 -48 -NULL 58 1969-12-31 16:00:01.148 58.0 NULL -58 58 0 58.0 0.0 -58.0 4614.074 0.0 -58.0 0.0 -10.175 58.0 -0.4531034482758621 58.0 0.0 58 -NULL -34 1969-12-31 16:00:04.695 -34.0 NULL 34 -34 0 -34.0 -0.0 34.0 -2704.8019999999997 0.0 34.0 0.0 -10.175 -34.0 0.7729411764705882 -34.0 0.0 -34 -NULL -59 1969-12-31 16:00:03.73 -59.0 NULL 59 -59 0 -59.0 -0.0 59.0 -4693.6269999999995 0.0 59.0 0.0 -10.175 -59.0 0.44542372881355935 -59.0 0.0 -59 -NULL -14 1969-12-31 16:00:08.549 -14.0 NULL 14 -14 0 -14.0 -0.0 14.0 -1113.742 0.0 14.0 0.0 -10.175 -14.0 1.8771428571428572 -14.0 0.0 -14 -NULL -11 1969-12-31 16:00:01.462 -11.0 NULL 11 -11 0 -11.0 -0.0 11.0 -875.083 0.0 11.0 0.0 -10.175 -11.0 2.389090909090909 -11.0 0.0 -11 -true 48 1969-12-31 16:00:06.604 48.0 1cGVWH7n1QU -48 48 0 48.0 0.0 -48.0 3818.544 0.0 -48.0 0.0 -10.175 48.0 -0.5475 48.0 0.0 48 -NULL -60 1969-12-31 16:00:02.322 -60.0 NULL 60 -60 0 -60.0 -0.0 60.0 -4773.18 0.0 60.0 0.0 -10.175 -60.0 0.438 -60.0 0.0 -60 -true 24 1969-12-31 16:00:08.983 24.0 cvLH6Eat2yFsyy7p -24 24 0 24.0 0.0 -24.0 1909.272 0.0 -24.0 0.0 -10.175 24.0 -1.095 24.0 0.0 24 -true 23 1969-12-31 16:00:13.439 23.0 821UdmGbkEf4j -23 23 0 23.0 0.0 -23.0 1829.719 0.0 -23.0 0.0 -10.175 23.0 -1.142608695652174 23.0 0.0 23 -true 55 1969-12-31 16:00:11.504 55.0 1cGVWH7n1QU -55 55 0 55.0 0.0 -55.0 4375.415 0.0 -55.0 0.0 -10.175 55.0 -0.47781818181818186 55.0 0.0 55 -NULL 44 1969-12-31 16:00:03.89 44.0 NULL -44 44 0 44.0 0.0 -44.0 3500.332 0.0 -44.0 0.0 -10.175 44.0 -0.5972727272727273 44.0 0.0 44 -NULL 51 1969-12-31 16:00:00.899 51.0 NULL -51 51 0 51.0 0.0 -51.0 4057.203 0.0 -51.0 0.0 -10.175 51.0 -0.5152941176470588 51.0 0.0 51 -NULL -33 1969-12-31 16:00:02.131 -33.0 NULL 33 -33 0 -33.0 -0.0 33.0 -2625.249 0.0 33.0 0.0 -10.175 -33.0 0.7963636363636364 -33.0 0.0 -33 -NULL 58 1969-12-31 16:00:03.366 58.0 NULL -58 58 0 58.0 0.0 -58.0 4614.074 0.0 -58.0 0.0 -10.175 58.0 -0.4531034482758621 58.0 0.0 58 -NULL -16 1969-12-31 16:00:02.239 -16.0 NULL 16 -16 0 -16.0 -0.0 16.0 -1272.848 0.0 16.0 0.0 -10.175 -16.0 1.6425 -16.0 0.0 -16 -true 36 1969-12-31 16:00:00.797 36.0 cvLH6Eat2yFsyy7p -36 36 0 36.0 0.0 -36.0 2863.908 0.0 -36.0 0.0 -10.175 36.0 -0.73 36.0 0.0 36 -NULL 45 1969-12-31 16:00:08.799 45.0 NULL -45 45 0 45.0 0.0 -45.0 3579.8849999999998 0.0 -45.0 0.0 -10.175 45.0 -0.5840000000000001 45.0 0.0 45 -NULL 32 1969-12-31 16:00:03.818 32.0 NULL -32 32 0 32.0 0.0 -32.0 2545.696 0.0 -32.0 0.0 -10.175 32.0 -0.82125 32.0 0.0 32 -true 0 1969-12-31 16:00:06.311 0.0 821UdmGbkEf4j 0 0 0 0.0 0.0 -0.0 0.0 0.0 -0.0 0.0 -10.175 0.0 NULL 0.0 NULL 0 -NULL 58 1969-12-31 16:00:13.801 58.0 NULL -58 58 0 58.0 0.0 -58.0 4614.074 0.0 -58.0 0.0 -10.175 58.0 -0.4531034482758621 58.0 0.0 58 -true 24 1969-12-31 16:00:00.723 24.0 1cGVWH7n1QU -24 24 0 24.0 0.0 -24.0 1909.272 0.0 -24.0 0.0 -10.175 24.0 -1.095 24.0 0.0 24 -true -17 1969-12-31 16:00:14.547 -17.0 cvLH6Eat2yFsyy7p 17 -17 0 -17.0 -0.0 17.0 -1352.4009999999998 0.0 17.0 0.0 -10.175 -17.0 1.5458823529411765 -17.0 0.0 -17 -NULL -28 1969-12-31 16:00:00.759 -28.0 NULL 28 -28 0 -28.0 -0.0 28.0 -2227.484 0.0 28.0 0.0 -10.175 -28.0 0.9385714285714286 -28.0 0.0 -28 -NULL -32 1969-12-31 16:00:12.413 -32.0 NULL 32 -32 0 -32.0 -0.0 32.0 -2545.696 0.0 32.0 0.0 -10.175 -32.0 0.82125 -32.0 0.0 -32 -true 57 1969-12-31 16:00:00.002 57.0 1cGVWH7n1QU -57 57 0 57.0 0.0 -57.0 4534.521 0.0 -57.0 0.0 -10.175 57.0 -0.4610526315789474 57.0 0.0 57 -NULL 22 1969-12-31 16:00:09.911 22.0 NULL -22 22 0 22.0 0.0 -22.0 1750.166 0.0 -22.0 0.0 -10.175 22.0 -1.1945454545454546 22.0 0.0 22 -NULL 57 1969-12-31 16:00:14.011 57.0 NULL -57 57 0 57.0 0.0 -57.0 4534.521 0.0 -57.0 0.0 -10.175 57.0 -0.4610526315789474 57.0 0.0 57 -true 59 1969-12-31 16:00:10.47 59.0 821UdmGbkEf4j -59 59 0 59.0 0.0 -59.0 4693.6269999999995 0.0 -59.0 0.0 -10.175 59.0 -0.44542372881355935 59.0 0.0 59 -true 14 1969-12-31 16:00:05.663 14.0 821UdmGbkEf4j -14 14 0 14.0 0.0 -14.0 1113.742 0.0 -14.0 0.0 -10.175 14.0 -1.8771428571428572 14.0 0.0 14 -NULL -50 1969-12-31 16:00:08.161 -50.0 NULL 50 -50 0 -50.0 -0.0 50.0 -3977.6499999999996 0.0 50.0 0.0 -10.175 -50.0 0.5256000000000001 -50.0 0.0 -50 -true 28 1969-12-31 16:00:06.969 28.0 1cGVWH7n1QU -28 28 0 28.0 0.0 -28.0 2227.484 0.0 -28.0 0.0 -10.175 28.0 -0.9385714285714286 28.0 0.0 28 -NULL 15 1969-12-31 16:00:15.923 15.0 NULL -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -NULL -1 1969-12-31 16:00:10.785 -1.0 NULL 1 -1 0 -1.0 -0.0 1.0 -79.553 0.0 1.0 0.0 -10.175 -1.0 26.28 -1.0 0.0 -1 -NULL -9 1969-12-31 16:00:03.03 -9.0 NULL 9 -9 0 -9.0 -0.0 9.0 -715.977 0.0 9.0 0.0 -10.175 -9.0 2.92 -9.0 0.0 -9 -true 16 1969-12-31 16:00:07.201 16.0 1cGVWH7n1QU -16 16 0 16.0 0.0 -16.0 1272.848 0.0 -16.0 0.0 -10.175 16.0 -1.6425 16.0 0.0 16 -NULL -37 1969-12-31 16:00:06.069 -37.0 NULL 37 -37 0 -37.0 -0.0 37.0 -2943.461 0.0 37.0 0.0 -10.175 -37.0 0.7102702702702703 -37.0 0.0 -37 -NULL 14 1969-12-31 16:00:10.307 14.0 NULL -14 14 0 14.0 0.0 -14.0 1113.742 0.0 -14.0 0.0 -10.175 14.0 -1.8771428571428572 14.0 0.0 14 -true -56 1969-12-31 16:00:00.65 -56.0 821UdmGbkEf4j 56 -56 0 -56.0 -0.0 56.0 -4454.968 0.0 56.0 0.0 -10.175 -56.0 0.4692857142857143 -56.0 0.0 -56 -NULL -32 1969-12-31 16:00:10.649 -32.0 NULL 32 -32 0 -32.0 -0.0 32.0 -2545.696 0.0 32.0 0.0 -10.175 -32.0 0.82125 -32.0 0.0 -32 -true -44 1969-12-31 16:00:04.472 -44.0 821UdmGbkEf4j 44 -44 0 -44.0 -0.0 44.0 -3500.332 0.0 44.0 0.0 -10.175 -44.0 0.5972727272727273 -44.0 0.0 -44 -true 36 1969-12-31 16:00:04.026 36.0 1cGVWH7n1QU -36 36 0 36.0 0.0 -36.0 2863.908 0.0 -36.0 0.0 -10.175 36.0 -0.73 36.0 0.0 36 -true -55 1969-12-31 16:00:12.297 -55.0 1cGVWH7n1QU 55 -55 0 -55.0 -0.0 55.0 -4375.415 0.0 55.0 0.0 -10.175 -55.0 0.47781818181818186 -55.0 0.0 -55 -true -57 1969-12-31 16:00:06.898 -57.0 821UdmGbkEf4j 57 -57 0 -57.0 -0.0 57.0 -4534.521 0.0 57.0 0.0 -10.175 -57.0 0.4610526315789474 -57.0 0.0 -57 -NULL 9 1969-12-31 16:00:03.557 9.0 NULL -9 9 0 9.0 0.0 -9.0 715.977 0.0 -9.0 0.0 -10.175 9.0 -2.92 9.0 0.0 9 -NULL -50 1969-12-31 16:00:05.159 -50.0 NULL 50 -50 0 -50.0 -0.0 50.0 -3977.6499999999996 0.0 50.0 0.0 -10.175 -50.0 0.5256000000000001 -50.0 0.0 -50 -NULL -45 1969-12-31 16:00:00.287 -45.0 NULL 45 -45 0 -45.0 -0.0 45.0 -3579.8849999999998 0.0 45.0 0.0 -10.175 -45.0 0.5840000000000001 -45.0 0.0 -45 -true 53 1969-12-31 16:00:05.393 53.0 821UdmGbkEf4j -53 53 0 53.0 0.0 -53.0 4216.309 0.0 -53.0 0.0 -10.175 53.0 -0.4958490566037736 53.0 0.0 53 -NULL -17 1969-12-31 16:00:13.479 -17.0 NULL 17 -17 0 -17.0 -0.0 17.0 -1352.4009999999998 0.0 17.0 0.0 -10.175 -17.0 1.5458823529411765 -17.0 0.0 -17 -NULL 20 1969-12-31 16:00:06.404 20.0 NULL -20 20 0 20.0 0.0 -20.0 1591.06 0.0 -20.0 0.0 -10.175 20.0 -1.314 20.0 0.0 20 -NULL -50 1969-12-31 16:00:00.568 -50.0 NULL 50 -50 0 -50.0 -0.0 50.0 -3977.6499999999996 0.0 50.0 0.0 -10.175 -50.0 0.5256000000000001 -50.0 0.0 -50 -true 51 1969-12-31 16:00:12.56 51.0 821UdmGbkEf4j -51 51 0 51.0 0.0 -51.0 4057.203 0.0 -51.0 0.0 -10.175 51.0 -0.5152941176470588 51.0 0.0 51 -NULL -44 1969-12-31 16:00:12.538 -44.0 NULL 44 -44 0 -44.0 -0.0 44.0 -3500.332 0.0 44.0 0.0 -10.175 -44.0 0.5972727272727273 -44.0 0.0 -44 -true -24 1969-12-31 16:00:02.676 -24.0 821UdmGbkEf4j 24 -24 0 -24.0 -0.0 24.0 -1909.272 0.0 24.0 0.0 -10.175 -24.0 1.095 -24.0 0.0 -24 -NULL 0 1969-12-31 16:00:14.343 0.0 NULL 0 0 0 0.0 0.0 -0.0 0.0 0.0 -0.0 0.0 -10.175 0.0 NULL 0.0 NULL 0 -true -22 1969-12-31 16:00:00.557 -22.0 cvLH6Eat2yFsyy7p 22 -22 0 -22.0 -0.0 22.0 -1750.166 0.0 22.0 0.0 -10.175 -22.0 1.1945454545454546 -22.0 0.0 -22 -true -9 1969-12-31 16:00:07.981 -9.0 1cGVWH7n1QU 9 -9 0 -9.0 -0.0 9.0 -715.977 0.0 9.0 0.0 -10.175 -9.0 2.92 -9.0 0.0 -9 -true 8 1969-12-31 16:00:08.893 8.0 1cGVWH7n1QU -8 8 0 8.0 0.0 -8.0 636.424 0.0 -8.0 0.0 -10.175 8.0 -3.285 8.0 0.0 8 -NULL -40 1969-12-31 16:00:06.489 -40.0 NULL 40 -40 0 -40.0 -0.0 40.0 -3182.12 0.0 40.0 0.0 -10.175 -40.0 0.657 -40.0 0.0 -40 -true 32 1969-12-31 16:00:07.085 32.0 cvLH6Eat2yFsyy7p -32 32 0 32.0 0.0 -32.0 2545.696 0.0 -32.0 0.0 -10.175 32.0 -0.82125 32.0 0.0 32 -true -9 1969-12-31 16:00:08.432 -9.0 1cGVWH7n1QU 9 -9 0 -9.0 -0.0 9.0 -715.977 0.0 9.0 0.0 -10.175 -9.0 2.92 -9.0 0.0 -9 -true 3 1969-12-31 16:00:07.913 3.0 1cGVWH7n1QU -3 3 0 3.0 0.0 -3.0 238.659 0.0 -3.0 0.0 -10.175 3.0 -8.76 3.0 0.0 3 -NULL 7 1969-12-31 16:00:05.724 7.0 NULL -7 7 0 7.0 0.0 -7.0 556.871 0.0 -7.0 0.0 -10.175 7.0 -3.7542857142857144 7.0 0.0 7 -NULL 22 1969-12-31 16:00:09.869 22.0 NULL -22 22 0 22.0 0.0 -22.0 1750.166 0.0 -22.0 0.0 -10.175 22.0 -1.1945454545454546 22.0 0.0 22 -true 60 1969-12-31 16:00:04.44 60.0 821UdmGbkEf4j -60 60 0 60.0 0.0 -60.0 4773.18 0.0 -60.0 0.0 -10.175 60.0 -0.438 60.0 0.0 60 -true -6 1969-12-31 16:00:11.511 -6.0 cvLH6Eat2yFsyy7p 6 -6 0 -6.0 -0.0 6.0 -477.318 0.0 6.0 0.0 -10.175 -6.0 4.38 -6.0 0.0 -6 -true 9 1969-12-31 16:00:00.377 9.0 cvLH6Eat2yFsyy7p -9 9 0 9.0 0.0 -9.0 715.977 0.0 -9.0 0.0 -10.175 9.0 -2.92 9.0 0.0 9 -NULL -1 1969-12-31 16:00:08.27 -1.0 NULL 1 -1 0 -1.0 -0.0 1.0 -79.553 0.0 1.0 0.0 -10.175 -1.0 26.28 -1.0 0.0 -1 -NULL 4 1969-12-31 16:00:11.153 4.0 NULL -4 4 0 4.0 0.0 -4.0 318.212 0.0 -4.0 0.0 -10.175 4.0 -6.57 4.0 0.0 4 -true 52 1969-12-31 16:00:06.246 52.0 cvLH6Eat2yFsyy7p -52 52 0 52.0 0.0 -52.0 4136.755999999999 0.0 -52.0 0.0 -10.175 52.0 -0.5053846153846154 52.0 0.0 52 -NULL 15 1969-12-31 16:00:11.526 15.0 NULL -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -true 30 1969-12-31 16:00:16.189 30.0 821UdmGbkEf4j -30 30 0 30.0 0.0 -30.0 2386.59 0.0 -30.0 0.0 -10.175 30.0 -0.876 30.0 0.0 30 -NULL -56 1969-12-31 16:00:03.199 -56.0 NULL 56 -56 0 -56.0 -0.0 56.0 -4454.968 0.0 56.0 0.0 -10.175 -56.0 0.4692857142857143 -56.0 0.0 -56 -true -43 1969-12-31 16:00:15.119 -43.0 821UdmGbkEf4j 43 -43 0 -43.0 -0.0 43.0 -3420.779 0.0 43.0 0.0 -10.175 -43.0 0.6111627906976744 -43.0 0.0 -43 -true 4 1969-12-31 16:00:02.212 4.0 1cGVWH7n1QU -4 4 0 4.0 0.0 -4.0 318.212 0.0 -4.0 0.0 -10.175 4.0 -6.57 4.0 0.0 4 -NULL 31 1969-12-31 16:00:01.849 31.0 NULL -31 31 0 31.0 0.0 -31.0 2466.143 0.0 -31.0 0.0 -10.175 31.0 -0.847741935483871 31.0 0.0 31 -true 48 1969-12-31 16:00:07.463 48.0 821UdmGbkEf4j -48 48 0 48.0 0.0 -48.0 3818.544 0.0 -48.0 0.0 -10.175 48.0 -0.5475 48.0 0.0 48 -true -38 1969-12-31 16:00:04.474 -38.0 1cGVWH7n1QU 38 -38 0 -38.0 -0.0 38.0 -3023.014 0.0 38.0 0.0 -10.175 -38.0 0.6915789473684211 -38.0 0.0 -38 -true -42 1969-12-31 16:00:15.646 -42.0 cvLH6Eat2yFsyy7p 42 -42 0 -42.0 -0.0 42.0 -3341.2259999999997 0.0 42.0 0.0 -10.175 -42.0 0.6257142857142858 -42.0 0.0 -42 -NULL 17 1969-12-31 16:00:07.703 17.0 NULL -17 17 0 17.0 0.0 -17.0 1352.4009999999998 0.0 -17.0 0.0 -10.175 17.0 -1.5458823529411765 17.0 0.0 17 -NULL -23 1969-12-31 16:00:15.004 -23.0 NULL 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -true -19 1969-12-31 16:00:14.471 -19.0 1cGVWH7n1QU 19 -19 0 -19.0 -0.0 19.0 -1511.507 0.0 19.0 0.0 -10.175 -19.0 1.3831578947368421 -19.0 0.0 -19 -true 57 1969-12-31 16:00:08.429 57.0 1cGVWH7n1QU -57 57 0 57.0 0.0 -57.0 4534.521 0.0 -57.0 0.0 -10.175 57.0 -0.4610526315789474 57.0 0.0 57 -NULL -44 1969-12-31 16:00:10.096 -44.0 NULL 44 -44 0 -44.0 -0.0 44.0 -3500.332 0.0 44.0 0.0 -10.175 -44.0 0.5972727272727273 -44.0 0.0 -44 -NULL -25 1969-12-31 16:00:07.022 -25.0 NULL 25 -25 0 -25.0 -0.0 25.0 -1988.8249999999998 0.0 25.0 0.0 -10.175 -25.0 1.0512000000000001 -25.0 0.0 -25 -true 59 1969-12-31 16:00:01.126 59.0 821UdmGbkEf4j -59 59 0 59.0 0.0 -59.0 4693.6269999999995 0.0 -59.0 0.0 -10.175 59.0 -0.44542372881355935 59.0 0.0 59 -NULL 42 1969-12-31 16:00:15.369 42.0 NULL -42 42 0 42.0 0.0 -42.0 3341.2259999999997 0.0 -42.0 0.0 -10.175 42.0 -0.6257142857142858 42.0 0.0 42 -true -47 1969-12-31 16:00:01.905 -47.0 1cGVWH7n1QU 47 -47 0 -47.0 -0.0 47.0 -3738.991 0.0 47.0 0.0 -10.175 -47.0 0.5591489361702128 -47.0 0.0 -47 -NULL -25 1969-12-31 16:00:06.199 -25.0 NULL 25 -25 0 -25.0 -0.0 25.0 -1988.8249999999998 0.0 25.0 0.0 -10.175 -25.0 1.0512000000000001 -25.0 0.0 -25 -true 38 1969-12-31 16:00:00.265 38.0 821UdmGbkEf4j -38 38 0 38.0 0.0 -38.0 3023.014 0.0 -38.0 0.0 -10.175 38.0 -0.6915789473684211 38.0 0.0 38 -true 15 1969-12-31 16:00:11.483 15.0 1cGVWH7n1QU -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -true 3 1969-12-31 16:00:13.741 3.0 cvLH6Eat2yFsyy7p -3 3 0 3.0 0.0 -3.0 238.659 0.0 -3.0 0.0 -10.175 3.0 -8.76 3.0 0.0 3 -true 25 1969-12-31 16:00:01.111 25.0 cvLH6Eat2yFsyy7p -25 25 0 25.0 0.0 -25.0 1988.8249999999998 0.0 -25.0 0.0 -10.175 25.0 -1.0512000000000001 25.0 0.0 25 -true 53 1969-12-31 16:00:10.607 53.0 1cGVWH7n1QU -53 53 0 53.0 0.0 -53.0 4216.309 0.0 -53.0 0.0 -10.175 53.0 -0.4958490566037736 53.0 0.0 53 -true 24 1969-12-31 16:00:06.905 24.0 1cGVWH7n1QU -24 24 0 24.0 0.0 -24.0 1909.272 0.0 -24.0 0.0 -10.175 24.0 -1.095 24.0 0.0 24 -true 26 1969-12-31 16:00:02.272 26.0 cvLH6Eat2yFsyy7p -26 26 0 26.0 0.0 -26.0 2068.3779999999997 0.0 -26.0 0.0 -10.175 26.0 -1.0107692307692309 26.0 0.0 26 -true -25 1969-12-31 16:00:07.101 -25.0 1cGVWH7n1QU 25 -25 0 -25.0 -0.0 25.0 -1988.8249999999998 0.0 25.0 0.0 -10.175 -25.0 1.0512000000000001 -25.0 0.0 -25 -true -46 1969-12-31 16:00:01.743 -46.0 cvLH6Eat2yFsyy7p 46 -46 0 -46.0 -0.0 46.0 -3659.438 0.0 46.0 0.0 -10.175 -46.0 0.571304347826087 -46.0 0.0 -46 -NULL 39 1969-12-31 16:00:02.789 39.0 NULL -39 39 0 39.0 0.0 -39.0 3102.567 0.0 -39.0 0.0 -10.175 39.0 -0.6738461538461539 39.0 0.0 39 -true 34 1969-12-31 16:00:01.639 34.0 cvLH6Eat2yFsyy7p -34 34 0 34.0 0.0 -34.0 2704.8019999999997 0.0 -34.0 0.0 -10.175 34.0 -0.7729411764705882 34.0 0.0 34 -true -21 1969-12-31 16:00:09.755 -21.0 1cGVWH7n1QU 21 -21 0 -21.0 -0.0 21.0 -1670.6129999999998 0.0 21.0 0.0 -10.175 -21.0 1.2514285714285716 -21.0 0.0 -21 -true 23 1969-12-31 16:00:09.059 23.0 821UdmGbkEf4j -23 23 0 23.0 0.0 -23.0 1829.719 0.0 -23.0 0.0 -10.175 23.0 -1.142608695652174 23.0 0.0 23 -NULL 15 1969-12-31 16:00:08.615 15.0 NULL -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -NULL 46 1969-12-31 16:00:09.963 46.0 NULL -46 46 0 46.0 0.0 -46.0 3659.438 0.0 -46.0 0.0 -10.175 46.0 -0.571304347826087 46.0 0.0 46 -NULL -30 1969-12-31 16:00:10.211 -30.0 NULL 30 -30 0 -30.0 -0.0 30.0 -2386.59 0.0 30.0 0.0 -10.175 -30.0 0.876 -30.0 0.0 -30 -NULL -18 1969-12-31 16:00:09.974 -18.0 NULL 18 -18 0 -18.0 -0.0 18.0 -1431.954 0.0 18.0 0.0 -10.175 -18.0 1.46 -18.0 0.0 -18 -NULL -11 1969-12-31 16:00:00.108 -11.0 NULL 11 -11 0 -11.0 -0.0 11.0 -875.083 0.0 11.0 0.0 -10.175 -11.0 2.389090909090909 -11.0 0.0 -11 -true 37 1969-12-31 16:00:02.443 37.0 821UdmGbkEf4j -37 37 0 37.0 0.0 -37.0 2943.461 0.0 -37.0 0.0 -10.175 37.0 -0.7102702702702703 37.0 0.0 37 -true 27 1969-12-31 16:00:09.093 27.0 1cGVWH7n1QU -27 27 0 27.0 0.0 -27.0 2147.931 0.0 -27.0 0.0 -10.175 27.0 -0.9733333333333334 27.0 0.0 27 -true -61 1969-12-31 16:00:15.325 -61.0 821UdmGbkEf4j 61 -61 0 -61.0 -0.0 61.0 -4852.733 0.0 61.0 0.0 -10.175 -61.0 0.4308196721311476 -61.0 0.0 -61 -true -12 1969-12-31 16:00:13.783 -12.0 cvLH6Eat2yFsyy7p 12 -12 0 -12.0 -0.0 12.0 -954.636 0.0 12.0 0.0 -10.175 -12.0 2.19 -12.0 0.0 -12 -true -3 1969-12-31 16:00:08.429 -3.0 821UdmGbkEf4j 3 -3 0 -3.0 -0.0 3.0 -238.659 0.0 3.0 0.0 -10.175 -3.0 8.76 -3.0 0.0 -3 -NULL 19 1969-12-31 16:00:15.68 19.0 NULL -19 19 0 19.0 0.0 -19.0 1511.507 0.0 -19.0 0.0 -10.175 19.0 -1.3831578947368421 19.0 0.0 19 -NULL 0 1969-12-31 16:00:04.626 0.0 NULL 0 0 0 0.0 0.0 -0.0 0.0 0.0 -0.0 0.0 -10.175 0.0 NULL 0.0 NULL 0 -NULL -53 1969-12-31 16:00:14.563 -53.0 NULL 53 -53 0 -53.0 -0.0 53.0 -4216.309 0.0 53.0 0.0 -10.175 -53.0 0.4958490566037736 -53.0 0.0 -53 -NULL -6 1969-12-31 16:00:06.4 -6.0 NULL 6 -6 0 -6.0 -0.0 6.0 -477.318 0.0 6.0 0.0 -10.175 -6.0 4.38 -6.0 0.0 -6 -NULL 43 1969-12-31 16:00:06.578 43.0 NULL -43 43 0 43.0 0.0 -43.0 3420.779 0.0 -43.0 0.0 -10.175 43.0 -0.6111627906976744 43.0 0.0 43 -NULL -28 1969-12-31 16:00:01.627 -28.0 NULL 28 -28 0 -28.0 -0.0 28.0 -2227.484 0.0 28.0 0.0 -10.175 -28.0 0.9385714285714286 -28.0 0.0 -28 -true -40 1969-12-31 16:00:03.802 -40.0 1cGVWH7n1QU 40 -40 0 -40.0 -0.0 40.0 -3182.12 0.0 40.0 0.0 -10.175 -40.0 0.657 -40.0 0.0 -40 -NULL 6 1969-12-31 16:00:01.308 6.0 NULL -6 6 0 6.0 0.0 -6.0 477.318 0.0 -6.0 0.0 -10.175 6.0 -4.38 6.0 0.0 6 -NULL -18 1969-12-31 16:00:00.564 -18.0 NULL 18 -18 0 -18.0 -0.0 18.0 -1431.954 0.0 18.0 0.0 -10.175 -18.0 1.46 -18.0 0.0 -18 -NULL -25 1969-12-31 16:00:15.769 -25.0 NULL 25 -25 0 -25.0 -0.0 25.0 -1988.8249999999998 0.0 25.0 0.0 -10.175 -25.0 1.0512000000000001 -25.0 0.0 -25 -true 46 1969-12-31 16:00:12.772 46.0 1cGVWH7n1QU -46 46 0 46.0 0.0 -46.0 3659.438 0.0 -46.0 0.0 -10.175 46.0 -0.571304347826087 46.0 0.0 46 -NULL -26 1969-12-31 16:00:14.191 -26.0 NULL 26 -26 0 -26.0 -0.0 26.0 -2068.3779999999997 0.0 26.0 0.0 -10.175 -26.0 1.0107692307692309 -26.0 0.0 -26 -NULL -7 1969-12-31 16:00:15.593 -7.0 NULL 7 -7 0 -7.0 -0.0 7.0 -556.871 0.0 7.0 0.0 -10.175 -7.0 3.7542857142857144 -7.0 0.0 -7 -true 61 1969-12-31 16:00:03.992 61.0 1cGVWH7n1QU -61 61 0 61.0 0.0 -61.0 4852.733 0.0 -61.0 0.0 -10.175 61.0 -0.4308196721311476 61.0 0.0 61 -NULL -62 1969-12-31 16:00:08.198 -62.0 NULL 62 -62 0 -62.0 -0.0 62.0 -4932.286 0.0 62.0 0.0 -10.175 -62.0 0.4238709677419355 -62.0 0.0 -62 -true 21 1969-12-31 16:00:02.216 21.0 821UdmGbkEf4j -21 21 0 21.0 0.0 -21.0 1670.6129999999998 0.0 -21.0 0.0 -10.175 21.0 -1.2514285714285716 21.0 0.0 21 -true 48 1969-12-31 16:00:03.11 48.0 1cGVWH7n1QU -48 48 0 48.0 0.0 -48.0 3818.544 0.0 -48.0 0.0 -10.175 48.0 -0.5475 48.0 0.0 48 -NULL -62 1969-12-31 16:00:01.22 -62.0 NULL 62 -62 0 -62.0 -0.0 62.0 -4932.286 0.0 62.0 0.0 -10.175 -62.0 0.4238709677419355 -62.0 0.0 -62 -NULL -46 1969-12-31 16:00:02.925 -46.0 NULL 46 -46 0 -46.0 -0.0 46.0 -3659.438 0.0 46.0 0.0 -10.175 -46.0 0.571304347826087 -46.0 0.0 -46 -true -62 1969-12-31 16:00:13.677 -62.0 cvLH6Eat2yFsyy7p 62 -62 0 -62.0 -0.0 62.0 -4932.286 0.0 62.0 0.0 -10.175 -62.0 0.4238709677419355 -62.0 0.0 -62 -true -13 1969-12-31 16:00:08.675 -13.0 cvLH6Eat2yFsyy7p 13 -13 0 -13.0 -0.0 13.0 -1034.1889999999999 0.0 13.0 0.0 -10.175 -13.0 2.0215384615384617 -13.0 0.0 -13 -true 0 1969-12-31 16:00:09.922 0.0 1cGVWH7n1QU 0 0 0 0.0 0.0 -0.0 0.0 0.0 -0.0 0.0 -10.175 0.0 NULL 0.0 NULL 0 -NULL 49 1969-12-31 16:00:10.324 49.0 NULL -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 -true 38 1969-12-31 16:00:08.278 38.0 1cGVWH7n1QU -38 38 0 38.0 0.0 -38.0 3023.014 0.0 -38.0 0.0 -10.175 38.0 -0.6915789473684211 38.0 0.0 38 -NULL -10 1969-12-31 16:00:08.386 -10.0 NULL 10 -10 0 -10.0 -0.0 10.0 -795.53 0.0 10.0 0.0 -10.175 -10.0 2.628 -10.0 0.0 -10 -true 24 1969-12-31 16:00:11.34 24.0 1cGVWH7n1QU -24 24 0 24.0 0.0 -24.0 1909.272 0.0 -24.0 0.0 -10.175 24.0 -1.095 24.0 0.0 24 -true -40 1969-12-31 16:00:05.804 -40.0 1cGVWH7n1QU 40 -40 0 -40.0 -0.0 40.0 -3182.12 0.0 40.0 0.0 -10.175 -40.0 0.657 -40.0 0.0 -40 -NULL 9 1969-12-31 16:00:11.148 9.0 NULL -9 9 0 9.0 0.0 -9.0 715.977 0.0 -9.0 0.0 -10.175 9.0 -2.92 9.0 0.0 9 -NULL 42 1969-12-31 16:00:05.092 42.0 NULL -42 42 0 42.0 0.0 -42.0 3341.2259999999997 0.0 -42.0 0.0 -10.175 42.0 -0.6257142857142858 42.0 0.0 42 -NULL -55 1969-12-31 16:00:11.38 -55.0 NULL 55 -55 0 -55.0 -0.0 55.0 -4375.415 0.0 55.0 0.0 -10.175 -55.0 0.47781818181818186 -55.0 0.0 -55 -true 53 1969-12-31 16:00:07.077 53.0 821UdmGbkEf4j -53 53 0 53.0 0.0 -53.0 4216.309 0.0 -53.0 0.0 -10.175 53.0 -0.4958490566037736 53.0 0.0 53 -true -10 1969-12-31 16:00:05.104 -10.0 1cGVWH7n1QU 10 -10 0 -10.0 -0.0 10.0 -795.53 0.0 10.0 0.0 -10.175 -10.0 2.628 -10.0 0.0 -10 -NULL 17 1969-12-31 16:00:04.513 17.0 NULL -17 17 0 17.0 0.0 -17.0 1352.4009999999998 0.0 -17.0 0.0 -10.175 17.0 -1.5458823529411765 17.0 0.0 17 -NULL 49 1969-12-31 16:00:10.523 49.0 NULL -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 -true -58 1969-12-31 16:00:02.61 -58.0 821UdmGbkEf4j 58 -58 0 -58.0 -0.0 58.0 -4614.074 0.0 58.0 0.0 -10.175 -58.0 0.4531034482758621 -58.0 0.0 -58 -NULL -49 1969-12-31 16:00:10.521 -49.0 NULL 49 -49 0 -49.0 -0.0 49.0 -3898.0969999999998 0.0 49.0 0.0 -10.175 -49.0 0.5363265306122449 -49.0 0.0 -49 -true 36 1969-12-31 16:00:15.145 36.0 1cGVWH7n1QU -36 36 0 36.0 0.0 -36.0 2863.908 0.0 -36.0 0.0 -10.175 36.0 -0.73 36.0 0.0 36 -true 31 1969-12-31 16:00:07.412 31.0 cvLH6Eat2yFsyy7p -31 31 0 31.0 0.0 -31.0 2466.143 0.0 -31.0 0.0 -10.175 31.0 -0.847741935483871 31.0 0.0 31 -NULL -1 1969-12-31 16:00:08.845 -1.0 NULL 1 -1 0 -1.0 -0.0 1.0 -79.553 0.0 1.0 0.0 -10.175 -1.0 26.28 -1.0 0.0 -1 -NULL 17 1969-12-31 16:00:05.162 17.0 NULL -17 17 0 17.0 0.0 -17.0 1352.4009999999998 0.0 -17.0 0.0 -10.175 17.0 -1.5458823529411765 17.0 0.0 17 -true -27 1969-12-31 16:00:13.68 -27.0 cvLH6Eat2yFsyy7p 27 -27 0 -27.0 -0.0 27.0 -2147.931 0.0 27.0 0.0 -10.175 -27.0 0.9733333333333334 -27.0 0.0 -27 -NULL -21 1969-12-31 16:00:14.72 -21.0 NULL 21 -21 0 -21.0 -0.0 21.0 -1670.6129999999998 0.0 21.0 0.0 -10.175 -21.0 1.2514285714285716 -21.0 0.0 -21 -true 55 1969-12-31 16:00:15.013 55.0 821UdmGbkEf4j -55 55 0 55.0 0.0 -55.0 4375.415 0.0 -55.0 0.0 -10.175 55.0 -0.47781818181818186 55.0 0.0 55 -NULL 52 1969-12-31 16:00:10.394 52.0 NULL -52 52 0 52.0 0.0 -52.0 4136.755999999999 0.0 -52.0 0.0 -10.175 52.0 -0.5053846153846154 52.0 0.0 52 -true -7 1969-12-31 16:00:04.292 -7.0 821UdmGbkEf4j 7 -7 0 -7.0 -0.0 7.0 -556.871 0.0 7.0 0.0 -10.175 -7.0 3.7542857142857144 -7.0 0.0 -7 -NULL -14 1969-12-31 16:00:04.887 -14.0 NULL 14 -14 0 -14.0 -0.0 14.0 -1113.742 0.0 14.0 0.0 -10.175 -14.0 1.8771428571428572 -14.0 0.0 -14 -true 1 1969-12-31 16:00:01.746 1.0 821UdmGbkEf4j -1 1 0 1.0 0.0 -1.0 79.553 0.0 -1.0 0.0 -10.175 1.0 -26.28 1.0 0.0 1 -true 38 1969-12-31 16:00:00.044 38.0 1cGVWH7n1QU -38 38 0 38.0 0.0 -38.0 3023.014 0.0 -38.0 0.0 -10.175 38.0 -0.6915789473684211 38.0 0.0 38 -NULL -4 1969-12-31 16:00:14.562 -4.0 NULL 4 -4 0 -4.0 -0.0 4.0 -318.212 0.0 4.0 0.0 -10.175 -4.0 6.57 -4.0 0.0 -4 -true 31 1969-12-31 16:00:07.021 31.0 cvLH6Eat2yFsyy7p -31 31 0 31.0 0.0 -31.0 2466.143 0.0 -31.0 0.0 -10.175 31.0 -0.847741935483871 31.0 0.0 31 -true 24 1969-12-31 16:00:00.294 24.0 1cGVWH7n1QU -24 24 0 24.0 0.0 -24.0 1909.272 0.0 -24.0 0.0 -10.175 24.0 -1.095 24.0 0.0 24 -NULL 20 1969-12-31 16:00:15.803 20.0 NULL -20 20 0 20.0 0.0 -20.0 1591.06 0.0 -20.0 0.0 -10.175 20.0 -1.314 20.0 0.0 20 -true -1 1969-12-31 16:00:02.578 -1.0 821UdmGbkEf4j 1 -1 0 -1.0 -0.0 1.0 -79.553 0.0 1.0 0.0 -10.175 -1.0 26.28 -1.0 0.0 -1 -NULL -56 1969-12-31 16:00:13.958 -56.0 NULL 56 -56 0 -56.0 -0.0 56.0 -4454.968 0.0 56.0 0.0 -10.175 -56.0 0.4692857142857143 -56.0 0.0 -56 -NULL 52 1969-12-31 16:00:02.269 52.0 NULL -52 52 0 52.0 0.0 -52.0 4136.755999999999 0.0 -52.0 0.0 -10.175 52.0 -0.5053846153846154 52.0 0.0 52 -NULL -21 1969-12-31 16:00:15.188 -21.0 NULL 21 -21 0 -21.0 -0.0 21.0 -1670.6129999999998 0.0 21.0 0.0 -10.175 -21.0 1.2514285714285716 -21.0 0.0 -21 -NULL -10 1969-12-31 16:00:07.925 -10.0 NULL 10 -10 0 -10.0 -0.0 10.0 -795.53 0.0 10.0 0.0 -10.175 -10.0 2.628 -10.0 0.0 -10 -true 32 1969-12-31 16:00:07.661 32.0 1cGVWH7n1QU -32 32 0 32.0 0.0 -32.0 2545.696 0.0 -32.0 0.0 -10.175 32.0 -0.82125 32.0 0.0 32 -true 32 1969-12-31 16:00:07.18 32.0 821UdmGbkEf4j -32 32 0 32.0 0.0 -32.0 2545.696 0.0 -32.0 0.0 -10.175 32.0 -0.82125 32.0 0.0 32 -true -52 1969-12-31 16:00:01.824 -52.0 821UdmGbkEf4j 52 -52 0 -52.0 -0.0 52.0 -4136.755999999999 0.0 52.0 0.0 -10.175 -52.0 0.5053846153846154 -52.0 0.0 -52 -NULL 6 1969-12-31 16:00:02.166 6.0 NULL -6 6 0 6.0 0.0 -6.0 477.318 0.0 -6.0 0.0 -10.175 6.0 -4.38 6.0 0.0 6 -NULL -37 1969-12-31 16:00:11.738 -37.0 NULL 37 -37 0 -37.0 -0.0 37.0 -2943.461 0.0 37.0 0.0 -10.175 -37.0 0.7102702702702703 -37.0 0.0 -37 -true -49 1969-12-31 16:00:04.575 -49.0 cvLH6Eat2yFsyy7p 49 -49 0 -49.0 -0.0 49.0 -3898.0969999999998 0.0 49.0 0.0 -10.175 -49.0 0.5363265306122449 -49.0 0.0 -49 -true 26 1969-12-31 16:00:10.31 26.0 821UdmGbkEf4j -26 26 0 26.0 0.0 -26.0 2068.3779999999997 0.0 -26.0 0.0 -10.175 26.0 -1.0107692307692309 26.0 0.0 26 -NULL 37 1969-12-31 16:00:00.051 37.0 NULL -37 37 0 37.0 0.0 -37.0 2943.461 0.0 -37.0 0.0 -10.175 37.0 -0.7102702702702703 37.0 0.0 37 -NULL -27 1969-12-31 16:00:15.435 -27.0 NULL 27 -27 0 -27.0 -0.0 27.0 -2147.931 0.0 27.0 0.0 -10.175 -27.0 0.9733333333333334 -27.0 0.0 -27 -NULL 15 1969-12-31 16:00:00.778 15.0 NULL -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -NULL 26 1969-12-31 16:00:00.521 26.0 NULL -26 26 0 26.0 0.0 -26.0 2068.3779999999997 0.0 -26.0 0.0 -10.175 26.0 -1.0107692307692309 26.0 0.0 26 -true -34 1969-12-31 16:00:00.987 -34.0 821UdmGbkEf4j 34 -34 0 -34.0 -0.0 34.0 -2704.8019999999997 0.0 34.0 0.0 -10.175 -34.0 0.7729411764705882 -34.0 0.0 -34 -NULL -1 1969-12-31 16:00:14.985 -1.0 NULL 1 -1 0 -1.0 -0.0 1.0 -79.553 0.0 1.0 0.0 -10.175 -1.0 26.28 -1.0 0.0 -1 -true -59 1969-12-31 16:00:11.758 -59.0 821UdmGbkEf4j 59 -59 0 -59.0 -0.0 59.0 -4693.6269999999995 0.0 59.0 0.0 -10.175 -59.0 0.44542372881355935 -59.0 0.0 -59 -true 38 1969-12-31 16:00:12.819 38.0 1cGVWH7n1QU -38 38 0 38.0 0.0 -38.0 3023.014 0.0 -38.0 0.0 -10.175 38.0 -0.6915789473684211 38.0 0.0 38 -NULL 52 1969-12-31 16:00:06.737 52.0 NULL -52 52 0 52.0 0.0 -52.0 4136.755999999999 0.0 -52.0 0.0 -10.175 52.0 -0.5053846153846154 52.0 0.0 52 -true -28 1969-12-31 16:00:11.675 -28.0 cvLH6Eat2yFsyy7p 28 -28 0 -28.0 -0.0 28.0 -2227.484 0.0 28.0 0.0 -10.175 -28.0 0.9385714285714286 -28.0 0.0 -28 -true 26 1969-12-31 16:00:05.384 26.0 1cGVWH7n1QU -26 26 0 26.0 0.0 -26.0 2068.3779999999997 0.0 -26.0 0.0 -10.175 26.0 -1.0107692307692309 26.0 0.0 26 -NULL 58 1969-12-31 16:00:12.924 58.0 NULL -58 58 0 58.0 0.0 -58.0 4614.074 0.0 -58.0 0.0 -10.175 58.0 -0.4531034482758621 58.0 0.0 58 -true 2 1969-12-31 16:00:12.4 2.0 821UdmGbkEf4j -2 2 0 2.0 0.0 -2.0 159.106 0.0 -2.0 0.0 -10.175 2.0 -13.14 2.0 0.0 2 -NULL 5 1969-12-31 16:00:14.201 5.0 NULL -5 5 0 5.0 0.0 -5.0 397.765 0.0 -5.0 0.0 -10.175 5.0 -5.256 5.0 0.0 5 -NULL -47 1969-12-31 16:00:13.73 -47.0 NULL 47 -47 0 -47.0 -0.0 47.0 -3738.991 0.0 47.0 0.0 -10.175 -47.0 0.5591489361702128 -47.0 0.0 -47 -NULL 41 1969-12-31 16:00:00.123 41.0 NULL -41 41 0 41.0 0.0 -41.0 3261.673 0.0 -41.0 0.0 -10.175 41.0 -0.6409756097560976 41.0 0.0 41 -NULL -11 1969-12-31 16:00:05.35 -11.0 NULL 11 -11 0 -11.0 -0.0 11.0 -875.083 0.0 11.0 0.0 -10.175 -11.0 2.389090909090909 -11.0 0.0 -11 -true 43 1969-12-31 16:00:14.133 43.0 cvLH6Eat2yFsyy7p -43 43 0 43.0 0.0 -43.0 3420.779 0.0 -43.0 0.0 -10.175 43.0 -0.6111627906976744 43.0 0.0 43 -true -22 1969-12-31 16:00:15.626 -22.0 cvLH6Eat2yFsyy7p 22 -22 0 -22.0 -0.0 22.0 -1750.166 0.0 22.0 0.0 -10.175 -22.0 1.1945454545454546 -22.0 0.0 -22 -NULL -23 1969-12-31 16:00:07.844 -23.0 NULL 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -NULL -24 1969-12-31 16:00:10.302 -24.0 NULL 24 -24 0 -24.0 -0.0 24.0 -1909.272 0.0 24.0 0.0 -10.175 -24.0 1.095 -24.0 0.0 -24 -true -29 1969-12-31 16:00:12.018 -29.0 1cGVWH7n1QU 29 -29 0 -29.0 -0.0 29.0 -2307.037 0.0 29.0 0.0 -10.175 -29.0 0.9062068965517242 -29.0 0.0 -29 -true 29 1969-12-31 16:00:06.958 29.0 cvLH6Eat2yFsyy7p -29 29 0 29.0 0.0 -29.0 2307.037 0.0 -29.0 0.0 -10.175 29.0 -0.9062068965517242 29.0 0.0 29 -NULL -55 1969-12-31 16:00:09.421 -55.0 NULL 55 -55 0 -55.0 -0.0 55.0 -4375.415 0.0 55.0 0.0 -10.175 -55.0 0.47781818181818186 -55.0 0.0 -55 -NULL -43 1969-12-31 16:00:01.901 -43.0 NULL 43 -43 0 -43.0 -0.0 43.0 -3420.779 0.0 43.0 0.0 -10.175 -43.0 0.6111627906976744 -43.0 0.0 -43 -NULL 51 1969-12-31 16:00:06.906 51.0 NULL -51 51 0 51.0 0.0 -51.0 4057.203 0.0 -51.0 0.0 -10.175 51.0 -0.5152941176470588 51.0 0.0 51 -true 21 1969-12-31 16:00:06.912 21.0 821UdmGbkEf4j -21 21 0 21.0 0.0 -21.0 1670.6129999999998 0.0 -21.0 0.0 -10.175 21.0 -1.2514285714285716 21.0 0.0 21 -NULL 10 1969-12-31 16:00:06.849 10.0 NULL -10 10 0 10.0 0.0 -10.0 795.53 0.0 -10.0 0.0 -10.175 10.0 -2.628 10.0 0.0 10 -true 10 1969-12-31 16:00:03.847 10.0 821UdmGbkEf4j -10 10 0 10.0 0.0 -10.0 795.53 0.0 -10.0 0.0 -10.175 10.0 -2.628 10.0 0.0 10 -NULL -44 1969-12-31 16:00:01.377 -44.0 NULL 44 -44 0 -44.0 -0.0 44.0 -3500.332 0.0 44.0 0.0 -10.175 -44.0 0.5972727272727273 -44.0 0.0 -44 -NULL 12 1969-12-31 16:00:12.608 12.0 NULL -12 12 0 12.0 0.0 -12.0 954.636 0.0 -12.0 0.0 -10.175 12.0 -2.19 12.0 0.0 12 -true 36 1969-12-31 16:00:13.825 36.0 cvLH6Eat2yFsyy7p -36 36 0 36.0 0.0 -36.0 2863.908 0.0 -36.0 0.0 -10.175 36.0 -0.73 36.0 0.0 36 -true -58 1969-12-31 16:00:14.933 -58.0 cvLH6Eat2yFsyy7p 58 -58 0 -58.0 -0.0 58.0 -4614.074 0.0 58.0 0.0 -10.175 -58.0 0.4531034482758621 -58.0 0.0 -58 -true -31 1969-12-31 16:00:06.79 -31.0 cvLH6Eat2yFsyy7p 31 -31 0 -31.0 -0.0 31.0 -2466.143 0.0 31.0 0.0 -10.175 -31.0 0.847741935483871 -31.0 0.0 -31 -NULL -11 1969-12-31 16:00:09.419 -11.0 NULL 11 -11 0 -11.0 -0.0 11.0 -875.083 0.0 11.0 0.0 -10.175 -11.0 2.389090909090909 -11.0 0.0 -11 -NULL -58 1969-12-31 16:00:12.683 -58.0 NULL 58 -58 0 -58.0 -0.0 58.0 -4614.074 0.0 58.0 0.0 -10.175 -58.0 0.4531034482758621 -58.0 0.0 -58 -true -23 1969-12-31 16:00:14.966 -23.0 cvLH6Eat2yFsyy7p 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -NULL -10 1969-12-31 16:00:13.613 -10.0 NULL 10 -10 0 -10.0 -0.0 10.0 -795.53 0.0 10.0 0.0 -10.175 -10.0 2.628 -10.0 0.0 -10 -NULL -3 1969-12-31 16:00:16.197 -3.0 NULL 3 -3 0 -3.0 -0.0 3.0 -238.659 0.0 3.0 0.0 -10.175 -3.0 8.76 -3.0 0.0 -3 -true 37 1969-12-31 16:00:14.907 37.0 cvLH6Eat2yFsyy7p -37 37 0 37.0 0.0 -37.0 2943.461 0.0 -37.0 0.0 -10.175 37.0 -0.7102702702702703 37.0 0.0 37 -NULL 20 1969-12-31 16:00:11.342 20.0 NULL -20 20 0 20.0 0.0 -20.0 1591.06 0.0 -20.0 0.0 -10.175 20.0 -1.314 20.0 0.0 20 -NULL -12 1969-12-31 16:00:03.275 -12.0 NULL 12 -12 0 -12.0 -0.0 12.0 -954.636 0.0 12.0 0.0 -10.175 -12.0 2.19 -12.0 0.0 -12 -true -62 1969-12-31 16:00:00.337 -62.0 1cGVWH7n1QU 62 -62 0 -62.0 -0.0 62.0 -4932.286 0.0 62.0 0.0 -10.175 -62.0 0.4238709677419355 -62.0 0.0 -62 -true -14 1969-12-31 16:00:00.82 -14.0 cvLH6Eat2yFsyy7p 14 -14 0 -14.0 -0.0 14.0 -1113.742 0.0 14.0 0.0 -10.175 -14.0 1.8771428571428572 -14.0 0.0 -14 -NULL -37 1969-12-31 16:00:13.801 -37.0 NULL 37 -37 0 -37.0 -0.0 37.0 -2943.461 0.0 37.0 0.0 -10.175 -37.0 0.7102702702702703 -37.0 0.0 -37 -NULL 1 1969-12-31 16:00:10.364 1.0 NULL -1 1 0 1.0 0.0 -1.0 79.553 0.0 -1.0 0.0 -10.175 1.0 -26.28 1.0 0.0 1 -true 61 1969-12-31 16:00:02.617 61.0 cvLH6Eat2yFsyy7p -61 61 0 61.0 0.0 -61.0 4852.733 0.0 -61.0 0.0 -10.175 61.0 -0.4308196721311476 61.0 0.0 61 -true 26 1969-12-31 16:00:07.972 26.0 821UdmGbkEf4j -26 26 0 26.0 0.0 -26.0 2068.3779999999997 0.0 -26.0 0.0 -10.175 26.0 -1.0107692307692309 26.0 0.0 26 -NULL -40 1969-12-31 16:00:02.171 -40.0 NULL 40 -40 0 -40.0 -0.0 40.0 -3182.12 0.0 40.0 0.0 -10.175 -40.0 0.657 -40.0 0.0 -40 -NULL 20 1969-12-31 16:00:08.368 20.0 NULL -20 20 0 20.0 0.0 -20.0 1591.06 0.0 -20.0 0.0 -10.175 20.0 -1.314 20.0 0.0 20 -NULL 54 1969-12-31 16:00:00.105 54.0 NULL -54 54 0 54.0 0.0 -54.0 4295.862 0.0 -54.0 0.0 -10.175 54.0 -0.4866666666666667 54.0 0.0 54 -NULL -4 1969-12-31 16:00:03.231 -4.0 NULL 4 -4 0 -4.0 -0.0 4.0 -318.212 0.0 4.0 0.0 -10.175 -4.0 6.57 -4.0 0.0 -4 -NULL 12 1969-12-31 16:00:00.038 12.0 NULL -12 12 0 12.0 0.0 -12.0 954.636 0.0 -12.0 0.0 -10.175 12.0 -2.19 12.0 0.0 12 -NULL 15 1969-12-31 16:00:10.415 15.0 NULL -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -true -60 1969-12-31 16:00:12.291 -60.0 821UdmGbkEf4j 60 -60 0 -60.0 -0.0 60.0 -4773.18 0.0 60.0 0.0 -10.175 -60.0 0.438 -60.0 0.0 -60 -true 30 1969-12-31 16:00:07.513 30.0 cvLH6Eat2yFsyy7p -30 30 0 30.0 0.0 -30.0 2386.59 0.0 -30.0 0.0 -10.175 30.0 -0.876 30.0 0.0 30 -NULL 42 1969-12-31 16:00:08.726 42.0 NULL -42 42 0 42.0 0.0 -42.0 3341.2259999999997 0.0 -42.0 0.0 -10.175 42.0 -0.6257142857142858 42.0 0.0 42 -true -52 1969-12-31 16:00:15.545 -52.0 821UdmGbkEf4j 52 -52 0 -52.0 -0.0 52.0 -4136.755999999999 0.0 52.0 0.0 -10.175 -52.0 0.5053846153846154 -52.0 0.0 -52 -NULL -10 1969-12-31 16:00:05.617 -10.0 NULL 10 -10 0 -10.0 -0.0 10.0 -795.53 0.0 10.0 0.0 -10.175 -10.0 2.628 -10.0 0.0 -10 -true -13 1969-12-31 16:00:08.499 -13.0 cvLH6Eat2yFsyy7p 13 -13 0 -13.0 -0.0 13.0 -1034.1889999999999 0.0 13.0 0.0 -10.175 -13.0 2.0215384615384617 -13.0 0.0 -13 -true 34 1969-12-31 16:00:07.178 34.0 cvLH6Eat2yFsyy7p -34 34 0 34.0 0.0 -34.0 2704.8019999999997 0.0 -34.0 0.0 -10.175 34.0 -0.7729411764705882 34.0 0.0 34 -true 22 1969-12-31 16:00:00.012 22.0 1cGVWH7n1QU -22 22 0 22.0 0.0 -22.0 1750.166 0.0 -22.0 0.0 -10.175 22.0 -1.1945454545454546 22.0 0.0 22 -true -35 1969-12-31 16:00:14.5 -35.0 1cGVWH7n1QU 35 -35 0 -35.0 -0.0 35.0 -2784.355 0.0 35.0 0.0 -10.175 -35.0 0.7508571428571429 -35.0 0.0 -35 -NULL -14 1969-12-31 16:00:08.704 -14.0 NULL 14 -14 0 -14.0 -0.0 14.0 -1113.742 0.0 14.0 0.0 -10.175 -14.0 1.8771428571428572 -14.0 0.0 -14 -NULL -56 1969-12-31 16:00:06.489 -56.0 NULL 56 -56 0 -56.0 -0.0 56.0 -4454.968 0.0 56.0 0.0 -10.175 -56.0 0.4692857142857143 -56.0 0.0 -56 -NULL 60 1969-12-31 16:00:06.811 60.0 NULL -60 60 0 60.0 0.0 -60.0 4773.18 0.0 -60.0 0.0 -10.175 60.0 -0.438 60.0 0.0 60 -NULL 4 1969-12-31 16:00:12.309 4.0 NULL -4 4 0 4.0 0.0 -4.0 318.212 0.0 -4.0 0.0 -10.175 4.0 -6.57 4.0 0.0 4 -NULL -19 1969-12-31 16:00:06.188 -19.0 NULL 19 -19 0 -19.0 -0.0 19.0 -1511.507 0.0 19.0 0.0 -10.175 -19.0 1.3831578947368421 -19.0 0.0 -19 -NULL -6 1969-12-31 16:00:08.781 -6.0 NULL 6 -6 0 -6.0 -0.0 6.0 -477.318 0.0 6.0 0.0 -10.175 -6.0 4.38 -6.0 0.0 -6 -NULL -24 1969-12-31 16:00:07.696 -24.0 NULL 24 -24 0 -24.0 -0.0 24.0 -1909.272 0.0 24.0 0.0 -10.175 -24.0 1.095 -24.0 0.0 -24 -true 15 1969-12-31 16:00:01.054 15.0 1cGVWH7n1QU -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -NULL -39 1969-12-31 16:00:07.721 -39.0 NULL 39 -39 0 -39.0 -0.0 39.0 -3102.567 0.0 39.0 0.0 -10.175 -39.0 0.6738461538461539 -39.0 0.0 -39 -true 14 1969-12-31 16:00:00.334 14.0 821UdmGbkEf4j -14 14 0 14.0 0.0 -14.0 1113.742 0.0 -14.0 0.0 -10.175 14.0 -1.8771428571428572 14.0 0.0 14 -NULL 33 1969-12-31 16:00:15.198 33.0 NULL -33 33 0 33.0 0.0 -33.0 2625.249 0.0 -33.0 0.0 -10.175 33.0 -0.7963636363636364 33.0 0.0 33 -true 35 1969-12-31 16:00:01.851 35.0 1cGVWH7n1QU -35 35 0 35.0 0.0 -35.0 2784.355 0.0 -35.0 0.0 -10.175 35.0 -0.7508571428571429 35.0 0.0 35 -NULL 0 1969-12-31 16:00:13.396 0.0 NULL 0 0 0 0.0 0.0 -0.0 0.0 0.0 -0.0 0.0 -10.175 0.0 NULL 0.0 NULL 0 -true 2 1969-12-31 16:00:02.359 2.0 cvLH6Eat2yFsyy7p -2 2 0 2.0 0.0 -2.0 159.106 0.0 -2.0 0.0 -10.175 2.0 -13.14 2.0 0.0 2 -true 40 1969-12-31 16:00:14.159 40.0 cvLH6Eat2yFsyy7p -40 40 0 40.0 0.0 -40.0 3182.12 0.0 -40.0 0.0 -10.175 40.0 -0.657 40.0 0.0 40 -NULL -58 1969-12-31 16:00:08.05 -58.0 NULL 58 -58 0 -58.0 -0.0 58.0 -4614.074 0.0 58.0 0.0 -10.175 -58.0 0.4531034482758621 -58.0 0.0 -58 -NULL -59 1969-12-31 16:00:13.15 -59.0 NULL 59 -59 0 -59.0 -0.0 59.0 -4693.6269999999995 0.0 59.0 0.0 -10.175 -59.0 0.44542372881355935 -59.0 0.0 -59 -true 37 1969-12-31 16:00:14.475 37.0 1cGVWH7n1QU -37 37 0 37.0 0.0 -37.0 2943.461 0.0 -37.0 0.0 -10.175 37.0 -0.7102702702702703 37.0 0.0 37 -NULL 55 1969-12-31 16:00:10.601 55.0 NULL -55 55 0 55.0 0.0 -55.0 4375.415 0.0 -55.0 0.0 -10.175 55.0 -0.47781818181818186 55.0 0.0 55 -true 11 1969-12-31 16:00:08.428 11.0 1cGVWH7n1QU -11 11 0 11.0 0.0 -11.0 875.083 0.0 -11.0 0.0 -10.175 11.0 -2.389090909090909 11.0 0.0 11 -true 39 1969-12-31 16:00:07.854 39.0 821UdmGbkEf4j -39 39 0 39.0 0.0 -39.0 3102.567 0.0 -39.0 0.0 -10.175 39.0 -0.6738461538461539 39.0 0.0 39 -true -33 1969-12-31 16:00:03.229 -33.0 821UdmGbkEf4j 33 -33 0 -33.0 -0.0 33.0 -2625.249 0.0 33.0 0.0 -10.175 -33.0 0.7963636363636364 -33.0 0.0 -33 -NULL -15 1969-12-31 16:00:07.485 -15.0 NULL 15 -15 0 -15.0 -0.0 15.0 -1193.295 0.0 15.0 0.0 -10.175 -15.0 1.752 -15.0 0.0 -15 -true 61 1969-12-31 16:00:02.688 61.0 1cGVWH7n1QU -61 61 0 61.0 0.0 -61.0 4852.733 0.0 -61.0 0.0 -10.175 61.0 -0.4308196721311476 61.0 0.0 61 -true 14 1969-12-31 16:00:03.058 14.0 cvLH6Eat2yFsyy7p -14 14 0 14.0 0.0 -14.0 1113.742 0.0 -14.0 0.0 -10.175 14.0 -1.8771428571428572 14.0 0.0 14 -NULL -13 1969-12-31 16:00:03.395 -13.0 NULL 13 -13 0 -13.0 -0.0 13.0 -1034.1889999999999 0.0 13.0 0.0 -10.175 -13.0 2.0215384615384617 -13.0 0.0 -13 -true 49 1969-12-31 16:00:15.523 49.0 cvLH6Eat2yFsyy7p -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 -NULL 39 1969-12-31 16:00:15.52 39.0 NULL -39 39 0 39.0 0.0 -39.0 3102.567 0.0 -39.0 0.0 -10.175 39.0 -0.6738461538461539 39.0 0.0 39 -NULL 48 1969-12-31 16:00:09.824 48.0 NULL -48 48 0 48.0 0.0 -48.0 3818.544 0.0 -48.0 0.0 -10.175 48.0 -0.5475 48.0 0.0 48 -NULL -5 1969-12-31 16:00:04.679 -5.0 NULL 5 -5 0 -5.0 -0.0 5.0 -397.765 0.0 5.0 0.0 -10.175 -5.0 5.256 -5.0 0.0 -5 -NULL -5 1969-12-31 16:00:08.267 -5.0 NULL 5 -5 0 -5.0 -0.0 5.0 -397.765 0.0 5.0 0.0 -10.175 -5.0 5.256 -5.0 0.0 -5 -NULL -54 1969-12-31 16:00:00.168 -54.0 NULL 54 -54 0 -54.0 -0.0 54.0 -4295.862 0.0 54.0 0.0 -10.175 -54.0 0.4866666666666667 -54.0 0.0 -54 -NULL -45 1969-12-31 16:00:11.856 -45.0 NULL 45 -45 0 -45.0 -0.0 45.0 -3579.8849999999998 0.0 45.0 0.0 -10.175 -45.0 0.5840000000000001 -45.0 0.0 -45 -NULL -34 1969-12-31 16:00:06.405 -34.0 NULL 34 -34 0 -34.0 -0.0 34.0 -2704.8019999999997 0.0 34.0 0.0 -10.175 -34.0 0.7729411764705882 -34.0 0.0 -34 -NULL -22 1969-12-31 16:00:05.996 -22.0 NULL 22 -22 0 -22.0 -0.0 22.0 -1750.166 0.0 22.0 0.0 -10.175 -22.0 1.1945454545454546 -22.0 0.0 -22 -true 52 1969-12-31 16:00:13.72 52.0 cvLH6Eat2yFsyy7p -52 52 0 52.0 0.0 -52.0 4136.755999999999 0.0 -52.0 0.0 -10.175 52.0 -0.5053846153846154 52.0 0.0 52 -true 11 1969-12-31 16:00:10.912 11.0 821UdmGbkEf4j -11 11 0 11.0 0.0 -11.0 875.083 0.0 -11.0 0.0 -10.175 11.0 -2.389090909090909 11.0 0.0 11 -NULL -33 1969-12-31 16:00:01.351 -33.0 NULL 33 -33 0 -33.0 -0.0 33.0 -2625.249 0.0 33.0 0.0 -10.175 -33.0 0.7963636363636364 -33.0 0.0 -33 -true 53 1969-12-31 16:00:11.254 53.0 cvLH6Eat2yFsyy7p -53 53 0 53.0 0.0 -53.0 4216.309 0.0 -53.0 0.0 -10.175 53.0 -0.4958490566037736 53.0 0.0 53 -true -10 1969-12-31 16:00:13.048 -10.0 1cGVWH7n1QU 10 -10 0 -10.0 -0.0 10.0 -795.53 0.0 10.0 0.0 -10.175 -10.0 2.628 -10.0 0.0 -10 -NULL -62 1969-12-31 16:00:12.591 -62.0 NULL 62 -62 0 -62.0 -0.0 62.0 -4932.286 0.0 62.0 0.0 -10.175 -62.0 0.4238709677419355 -62.0 0.0 -62 -NULL -4 1969-12-31 16:00:06.776 -4.0 NULL 4 -4 0 -4.0 -0.0 4.0 -318.212 0.0 4.0 0.0 -10.175 -4.0 6.57 -4.0 0.0 -4 -true 10 1969-12-31 16:00:12.588 10.0 1cGVWH7n1QU -10 10 0 10.0 0.0 -10.0 795.53 0.0 -10.0 0.0 -10.175 10.0 -2.628 10.0 0.0 10 -true -29 1969-12-31 16:00:08.063 -29.0 1cGVWH7n1QU 29 -29 0 -29.0 -0.0 29.0 -2307.037 0.0 29.0 0.0 -10.175 -29.0 0.9062068965517242 -29.0 0.0 -29 -NULL -14 1969-12-31 16:00:09.123 -14.0 NULL 14 -14 0 -14.0 -0.0 14.0 -1113.742 0.0 14.0 0.0 -10.175 -14.0 1.8771428571428572 -14.0 0.0 -14 -NULL -39 1969-12-31 16:00:13.092 -39.0 NULL 39 -39 0 -39.0 -0.0 39.0 -3102.567 0.0 39.0 0.0 -10.175 -39.0 0.6738461538461539 -39.0 0.0 -39 -true -13 1969-12-31 16:00:00.457 -13.0 cvLH6Eat2yFsyy7p 13 -13 0 -13.0 -0.0 13.0 -1034.1889999999999 0.0 13.0 0.0 -10.175 -13.0 2.0215384615384617 -13.0 0.0 -13 -true 46 1969-12-31 16:00:06.626 46.0 cvLH6Eat2yFsyy7p -46 46 0 46.0 0.0 -46.0 3659.438 0.0 -46.0 0.0 -10.175 46.0 -0.571304347826087 46.0 0.0 46 -true 5 1969-12-31 16:00:00.344 5.0 1cGVWH7n1QU -5 5 0 5.0 0.0 -5.0 397.765 0.0 -5.0 0.0 -10.175 5.0 -5.256 5.0 0.0 5 -NULL 17 1969-12-31 16:00:13.411 17.0 NULL -17 17 0 17.0 0.0 -17.0 1352.4009999999998 0.0 -17.0 0.0 -10.175 17.0 -1.5458823529411765 17.0 0.0 17 -NULL -23 1969-12-31 16:00:06.523 -23.0 NULL 23 -23 0 -23.0 -0.0 23.0 -1829.719 0.0 23.0 0.0 -10.175 -23.0 1.142608695652174 -23.0 0.0 -23 -true -12 1969-12-31 16:00:13.178 -12.0 1cGVWH7n1QU 12 -12 0 -12.0 -0.0 12.0 -954.636 0.0 12.0 0.0 -10.175 -12.0 2.19 -12.0 0.0 -12 -true 51 1969-12-31 16:00:08.804 51.0 1cGVWH7n1QU -51 51 0 51.0 0.0 -51.0 4057.203 0.0 -51.0 0.0 -10.175 51.0 -0.5152941176470588 51.0 0.0 51 -true 45 1969-12-31 16:00:11.503 45.0 1cGVWH7n1QU -45 45 0 45.0 0.0 -45.0 3579.8849999999998 0.0 -45.0 0.0 -10.175 45.0 -0.5840000000000001 45.0 0.0 45 -NULL 28 1969-12-31 16:00:02.344 28.0 NULL -28 28 0 28.0 0.0 -28.0 2227.484 0.0 -28.0 0.0 -10.175 28.0 -0.9385714285714286 28.0 0.0 28 -true -44 1969-12-31 16:00:08.744 -44.0 cvLH6Eat2yFsyy7p 44 -44 0 -44.0 -0.0 44.0 -3500.332 0.0 44.0 0.0 -10.175 -44.0 0.5972727272727273 -44.0 0.0 -44 -true -47 1969-12-31 16:00:02.527 -47.0 1cGVWH7n1QU 47 -47 0 -47.0 -0.0 47.0 -3738.991 0.0 47.0 0.0 -10.175 -47.0 0.5591489361702128 -47.0 0.0 -47 -true 10 1969-12-31 16:00:12.034 10.0 821UdmGbkEf4j -10 10 0 10.0 0.0 -10.0 795.53 0.0 -10.0 0.0 -10.175 10.0 -2.628 10.0 0.0 10 -NULL 0 1969-12-31 16:00:14.822 0.0 NULL 0 0 0 0.0 0.0 -0.0 0.0 0.0 -0.0 0.0 -10.175 0.0 NULL 0.0 NULL 0 -NULL 8 1969-12-31 16:00:15.683 8.0 NULL -8 8 0 8.0 0.0 -8.0 636.424 0.0 -8.0 0.0 -10.175 8.0 -3.285 8.0 0.0 8 -NULL 61 1969-12-31 16:00:01.091 61.0 NULL -61 61 0 61.0 0.0 -61.0 4852.733 0.0 -61.0 0.0 -10.175 61.0 -0.4308196721311476 61.0 0.0 61 -true 60 1969-12-31 16:00:10.065 60.0 821UdmGbkEf4j -60 60 0 60.0 0.0 -60.0 4773.18 0.0 -60.0 0.0 -10.175 60.0 -0.438 60.0 0.0 60 -NULL -27 1969-12-31 16:00:09.782 -27.0 NULL 27 -27 0 -27.0 -0.0 27.0 -2147.931 0.0 27.0 0.0 -10.175 -27.0 0.9733333333333334 -27.0 0.0 -27 -true 32 1969-12-31 16:00:11.054 32.0 cvLH6Eat2yFsyy7p -32 32 0 32.0 0.0 -32.0 2545.696 0.0 -32.0 0.0 -10.175 32.0 -0.82125 32.0 0.0 32 -NULL 37 1969-12-31 16:00:09.907 37.0 NULL -37 37 0 37.0 0.0 -37.0 2943.461 0.0 -37.0 0.0 -10.175 37.0 -0.7102702702702703 37.0 0.0 37 -NULL -21 1969-12-31 16:00:05.635 -21.0 NULL 21 -21 0 -21.0 -0.0 21.0 -1670.6129999999998 0.0 21.0 0.0 -10.175 -21.0 1.2514285714285716 -21.0 0.0 -21 -NULL -47 1969-12-31 16:00:08.091 -47.0 NULL 47 -47 0 -47.0 -0.0 47.0 -3738.991 0.0 47.0 0.0 -10.175 -47.0 0.5591489361702128 -47.0 0.0 -47 -true 12 1969-12-31 16:00:04.811 12.0 cvLH6Eat2yFsyy7p -12 12 0 12.0 0.0 -12.0 954.636 0.0 -12.0 0.0 -10.175 12.0 -2.19 12.0 0.0 12 -NULL 14 1969-12-31 16:00:03.834 14.0 NULL -14 14 0 14.0 0.0 -14.0 1113.742 0.0 -14.0 0.0 -10.175 14.0 -1.8771428571428572 14.0 0.0 14 -true -27 1969-12-31 16:00:04.008 -27.0 cvLH6Eat2yFsyy7p 27 -27 0 -27.0 -0.0 27.0 -2147.931 0.0 27.0 0.0 -10.175 -27.0 0.9733333333333334 -27.0 0.0 -27 -true 53 1969-12-31 16:00:16.36 53.0 1cGVWH7n1QU -53 53 0 53.0 0.0 -53.0 4216.309 0.0 -53.0 0.0 -10.175 53.0 -0.4958490566037736 53.0 0.0 53 -NULL 52 1969-12-31 16:00:00.072 52.0 NULL -52 52 0 52.0 0.0 -52.0 4136.755999999999 0.0 -52.0 0.0 -10.175 52.0 -0.5053846153846154 52.0 0.0 52 -NULL 48 1969-12-31 16:00:07.115 48.0 NULL -48 48 0 48.0 0.0 -48.0 3818.544 0.0 -48.0 0.0 -10.175 48.0 -0.5475 48.0 0.0 48 -true 35 1969-12-31 16:00:11.763 35.0 1cGVWH7n1QU -35 35 0 35.0 0.0 -35.0 2784.355 0.0 -35.0 0.0 -10.175 35.0 -0.7508571428571429 35.0 0.0 35 -NULL 49 1969-12-31 16:00:00.578 49.0 NULL -49 49 0 49.0 0.0 -49.0 3898.0969999999998 0.0 -49.0 0.0 -10.175 49.0 -0.5363265306122449 49.0 0.0 49 -NULL -30 1969-12-31 16:00:15.136 -30.0 NULL 30 -30 0 -30.0 -0.0 30.0 -2386.59 0.0 30.0 0.0 -10.175 -30.0 0.876 -30.0 0.0 -30 -true -50 1969-12-31 16:00:08.625 -50.0 cvLH6Eat2yFsyy7p 50 -50 0 -50.0 -0.0 50.0 -3977.6499999999996 0.0 50.0 0.0 -10.175 -50.0 0.5256000000000001 -50.0 0.0 -50 -NULL 25 1969-12-31 16:00:11.681 25.0 NULL -25 25 0 25.0 0.0 -25.0 1988.8249999999998 0.0 -25.0 0.0 -10.175 25.0 -1.0512000000000001 25.0 0.0 25 -true -29 1969-12-31 16:00:05.209 -29.0 1cGVWH7n1QU 29 -29 0 -29.0 -0.0 29.0 -2307.037 0.0 29.0 0.0 -10.175 -29.0 0.9062068965517242 -29.0 0.0 -29 -true 50 1969-12-31 16:00:09.868 50.0 1cGVWH7n1QU -50 50 0 50.0 0.0 -50.0 3977.6499999999996 0.0 -50.0 0.0 -10.175 50.0 -0.5256000000000001 50.0 0.0 50 -true 7 1969-12-31 16:00:11.886 7.0 cvLH6Eat2yFsyy7p -7 7 0 7.0 0.0 -7.0 556.871 0.0 -7.0 0.0 -10.175 7.0 -3.7542857142857144 7.0 0.0 7 -NULL 15 1969-12-31 16:00:00.766 15.0 NULL -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -NULL 57 1969-12-31 16:00:03.066 57.0 NULL -57 57 0 57.0 0.0 -57.0 4534.521 0.0 -57.0 0.0 -10.175 57.0 -0.4610526315789474 57.0 0.0 57 -NULL 16 1969-12-31 16:00:07.678 16.0 NULL -16 16 0 16.0 0.0 -16.0 1272.848 0.0 -16.0 0.0 -10.175 16.0 -1.6425 16.0 0.0 16 -NULL 22 1969-12-31 16:00:12.747 22.0 NULL -22 22 0 22.0 0.0 -22.0 1750.166 0.0 -22.0 0.0 -10.175 22.0 -1.1945454545454546 22.0 0.0 22 -NULL -49 1969-12-31 16:00:05.83 -49.0 NULL 49 -49 0 -49.0 -0.0 49.0 -3898.0969999999998 0.0 49.0 0.0 -10.175 -49.0 0.5363265306122449 -49.0 0.0 -49 -true -29 1969-12-31 16:00:05.977 -29.0 821UdmGbkEf4j 29 -29 0 -29.0 -0.0 29.0 -2307.037 0.0 29.0 0.0 -10.175 -29.0 0.9062068965517242 -29.0 0.0 -29 -NULL 40 1969-12-31 16:00:14.048 40.0 NULL -40 40 0 40.0 0.0 -40.0 3182.12 0.0 -40.0 0.0 -10.175 40.0 -0.657 40.0 0.0 40 -NULL 57 1969-12-31 16:00:01.811 57.0 NULL -57 57 0 57.0 0.0 -57.0 4534.521 0.0 -57.0 0.0 -10.175 57.0 -0.4610526315789474 57.0 0.0 57 -NULL -40 1969-12-31 16:00:10.027 -40.0 NULL 40 -40 0 -40.0 -0.0 40.0 -3182.12 0.0 40.0 0.0 -10.175 -40.0 0.657 -40.0 0.0 -40 -true 15 1969-12-31 16:00:00.019 15.0 cvLH6Eat2yFsyy7p -15 15 0 15.0 0.0 -15.0 1193.295 0.0 -15.0 0.0 -10.175 15.0 -1.752 15.0 0.0 15 -true 3 1969-12-31 16:00:08.876 3.0 821UdmGbkEf4j -3 3 0 3.0 0.0 -3.0 238.659 0.0 -3.0 0.0 -10.175 3.0 -8.76 3.0 0.0 3 -NULL 4 1969-12-31 16:00:00.308 4.0 NULL -4 4 0 4.0 0.0 -4.0 318.212 0.0 -4.0 0.0 -10.175 4.0 -6.57 4.0 0.0 4 -NULL -6 1969-12-31 16:00:01.153 -6.0 NULL 6 -6 0 -6.0 -0.0 6.0 -477.318 0.0 6.0 0.0 -10.175 -6.0 4.38 -6.0 0.0 -6 -true -57 1969-12-31 16:00:10.312 -57.0 821UdmGbkEf4j 57 -57 0 -57.0 -0.0 57.0 -4534.521 0.0 57.0 0.0 -10.175 -57.0 0.4610526315789474 -57.0 0.0 -57 -true -57 1969-12-31 16:00:05.318 -57.0 1cGVWH7n1QU 57 -57 0 -57.0 -0.0 57.0 -4534.521 0.0 57.0 0.0 -10.175 -57.0 0.4610526315789474 -57.0 0.0 -57 -NULL -5 1969-12-31 16:00:10.326 -5.0 NULL 5 -5 0 -5.0 -0.0 5.0 -397.765 0.0 5.0 0.0 -10.175 -5.0 5.256 -5.0 0.0 -5 -NULL -59 1969-12-31 16:00:15.861 -59.0 NULL 59 -59 0 -59.0 -0.0 59.0 -4693.6269999999995 0.0 59.0 0.0 -10.175 -59.0 0.44542372881355935 -59.0 0.0 -59 -NULL -18 1969-12-31 16:00:04.625 -18.0 NULL 18 -18 0 -18.0 -0.0 18.0 -1431.954 0.0 18.0 0.0 -10.175 -18.0 1.46 -18.0 0.0 -18 -NULL -33 1969-12-31 16:00:14.257 -33.0 NULL 33 -33 0 -33.0 -0.0 33.0 -2625.249 0.0 33.0 0.0 -10.175 -33.0 0.7963636363636364 -33.0 0.0 -33 -true -55 1969-12-31 16:00:05.568 -55.0 cvLH6Eat2yFsyy7p 55 -55 0 -55.0 -0.0 55.0 -4375.415 0.0 55.0 0.0 -10.175 -55.0 0.47781818181818186 -55.0 0.0 -55 -true 16 1969-12-31 16:00:01.151 16.0 821UdmGbkEf4j -16 16 0 16.0 0.0 -16.0 1272.848 0.0 -16.0 0.0 -10.175 16.0 -1.6425 16.0 0.0 16 diff --git ql/src/test/results/clientpositive/spark/vectorization_decimal_date.q.out ql/src/test/results/clientpositive/spark/vectorization_decimal_date.q.out index 067f183..19799b3 100644 --- ql/src/test/results/clientpositive/spark/vectorization_decimal_date.q.out +++ ql/src/test/results/clientpositive/spark/vectorization_decimal_date.q.out @@ -3,7 +3,6 @@ PREHOOK: type: CREATETABLE_AS_SELECT PREHOOK: Input: default@alltypesorc PREHOOK: Output: database:default PREHOOK: Output: default@date_decimal_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: CREATE TABLE date_decimal_test STORED AS ORC AS SELECT cint, cdouble, CAST (CAST (cint AS TIMESTAMP) AS DATE) AS cdate, CAST (((cdouble*22.1)/37) AS DECIMAL(20,10)) AS cdecimal FROM alltypesorc POSTHOOK: type: CREATETABLE_AS_SELECT POSTHOOK: Input: default@alltypesorc @@ -23,17 +22,17 @@ STAGE PLANS: Processor Tree: TableScan alias: date_decimal_test - Statistics: Num rows: 482 Data size: 86777 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 12288 Data size: 1651260 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (cint is not null and cdouble is not null) (type: boolean) - Statistics: Num rows: 121 Data size: 21784 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3072 Data size: 412815 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: cdate (type: date), cdecimal (type: decimal(20,10)) outputColumnNames: _col0, _col1 - Statistics: Num rows: 121 Data size: 21784 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3072 Data size: 412815 Basic stats: COMPLETE Column stats: NONE Limit Number of rows: 10 - Statistics: Num rows: 10 Data size: 1800 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 10 Data size: 1340 Basic stats: COMPLETE Column stats: NONE ListSink PREHOOK: query: SELECT cdate, cdecimal from date_decimal_test where cint IS NOT NULL AND cdouble IS NOT NULL LIMIT 10 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,