diff --git common/src/java/org/apache/hadoop/hive/conf/HiveConf.java common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index 27f68df..736a22d 100644 --- common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -2187,6 +2187,13 @@ public void setSparkConfigUpdated(boolean isSparkConfigUpdated) { "Channel logging level for remote Spark driver. One of {DEBUG, ERROR, INFO, TRACE, WARN}."), SPARK_RPC_SASL_MECHANISM("hive.spark.client.rpc.sasl.mechanisms", "DIGEST-MD5", "Name of the SASL mechanism to use for authentication."), + SPARK_DYNAMIC_PARTITION_PRUNING( + "hive.spark.dynamic.partition.pruning", false, + "When dynamic pruning is enabled, joins on partition keys will be processed by writing\n" + + "to a temporary HDFS file, and read later for removing unnecessary partitions."), + SPARK_DYNAMIC_PARTITION_PRUNING_MAX_DATA_SIZE( + "hive.spark.dynamic.partition.pruning.max.data.size", 100*1024*1024L, + "Maximum total data size in dynamic pruning."), NWAYJOINREORDER("hive.reorder.nway.joins", true, "Runs reordering of tables within single n-way join (i.e.: picks streamtable)"), HIVE_LOG_N_RECORDS("hive.log.every.n.records", 0L, new RangeValidator(0L, null), diff --git itests/src/test/resources/testconfiguration.properties itests/src/test/resources/testconfiguration.properties index f4ae684..ca5c789 100644 --- itests/src/test/resources/testconfiguration.properties +++ itests/src/test/resources/testconfiguration.properties @@ -953,6 +953,9 @@ spark.query.files=add_part_multiple.q, \ smb_mapjoin_8.q, \ smb_mapjoin_9.q, \ sort.q, \ + spark_dynamic_partition_pruning.q, \ + spark_dynamic_partition_pruning_2.q, \ + spark_vectorized_dynamic_partition_pruning.q, \ stats0.q, \ stats1.q, \ stats10.q, \ diff --git ql/if/queryplan.thrift ql/if/queryplan.thrift index c8dfa35..cc9af94 100644 --- ql/if/queryplan.thrift +++ ql/if/queryplan.thrift @@ -60,6 +60,7 @@ enum OperatorType { ORCFILEMERGE, RCFILEMERGE, MERGEJOIN, + SPARKPRUNINGSINK, } struct Operator { diff --git ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/OperatorType.java ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/OperatorType.java index e18f935..4dfaa22 100644 --- ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/OperatorType.java +++ ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/OperatorType.java @@ -36,7 +36,8 @@ EVENT(21), ORCFILEMERGE(22), RCFILEMERGE(23), - MERGEJOIN(24); + MERGEJOIN(24), + SPARKPRUNINGSINK(25); private final int value; @@ -107,6 +108,8 @@ public static OperatorType findByValue(int value) { return RCFILEMERGE; case 24: return MERGEJOIN; + case 25: + return SPARKPRUNINGSINK; default: return null; } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java index f58a10b..ff58741 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java @@ -34,8 +34,11 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorSMBMapJoinOperator; import org.apache.hadoop.hive.ql.exec.vector.VectorSelectOperator; import org.apache.hadoop.hive.ql.exec.vector.VectorSparkHashTableSinkOperator; +import org.apache.hadoop.hive.ql.exec.vector.VectorSparkPartitionPruningSinkOperator; import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext; import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.optimizer.spark.SparkPartitionPruningSinkDesc; +import org.apache.hadoop.hive.ql.parse.spark.SparkPartitionPruningSinkOperator; import org.apache.hadoop.hive.ql.plan.AppMasterEventDesc; import org.apache.hadoop.hive.ql.plan.CollectDesc; import org.apache.hadoop.hive.ql.plan.CommonMergeJoinDesc; @@ -117,6 +120,8 @@ AppMasterEventOperator.class)); opvec.add(new OpTuple(DynamicPruningEventDesc.class, AppMasterEventOperator.class)); + opvec.add(new OpTuple(SparkPartitionPruningSinkDesc.class, + SparkPartitionPruningSinkOperator.class)); opvec.add(new OpTuple(RCFileMergeDesc.class, RCFileMergeOperator.class)); opvec.add(new OpTuple(OrcFileMergeDesc.class, @@ -133,6 +138,9 @@ VectorAppMasterEventOperator.class)); vectorOpvec.add(new OpTuple(DynamicPruningEventDesc.class, VectorAppMasterEventOperator.class)); + vectorOpvec.add(new OpTuple( + SparkPartitionPruningSinkDesc.class, + VectorSparkPartitionPruningSinkOperator.class)); vectorOpvec.add(new OpTuple(SelectDesc.class, VectorSelectOperator.class)); vectorOpvec.add(new OpTuple(GroupByDesc.class, VectorGroupByOperator.class)); vectorOpvec.add(new OpTuple(MapJoinDesc.class, VectorMapJoinOperator.class)); diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java index 21398d8..007db75 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java @@ -33,6 +33,7 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; import org.apache.hadoop.hive.ql.io.HiveKey; import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.Writable; import org.apache.hive.spark.client.rpc.RpcConfiguration; import org.apache.spark.SparkConf; import org.apache.spark.SparkException; @@ -148,11 +149,11 @@ public static HiveSparkClient createHiveSparkClient(HiveConf hiveconf) Set classes = Sets.newHashSet( Splitter.on(",").trimResults().omitEmptyStrings().split( Strings.nullToEmpty(sparkConf.get("spark.kryo.classesToRegister")))); + classes.add(Writable.class.getName()); classes.add(VectorizedRowBatch.class.getName()); classes.add(BytesWritable.class.getName()); classes.add(HiveKey.class.getName()); - sparkConf.put( - "spark.kryo.classesToRegister", Joiner.on(",").join(classes)); + sparkConf.put("spark.kryo.classesToRegister", Joiner.on(",").join(classes)); return sparkConf; } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkDynamicPartitionPruner.java ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkDynamicPartitionPruner.java new file mode 100644 index 0000000..52913e0 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkDynamicPartitionPruner.java @@ -0,0 +1,268 @@ +/** + * 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; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import com.clearspring.analytics.util.Preconditions; +import javolution.testing.AssertionException; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator; +import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluatorFactory; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.MapWork; +import org.apache.hadoop.hive.ql.plan.PartitionDesc; +import org.apache.hadoop.hive.ql.plan.TableDesc; +import org.apache.hadoop.hive.serde2.Deserializer; +import org.apache.hadoop.hive.serde2.SerDeException; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils; +import org.apache.hadoop.hive.serde2.objectinspector.StructField; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.util.ReflectionUtils; + +/** + * The spark version of DynamicPartitionPruner. + */ +public class SparkDynamicPartitionPruner { + private static final Log LOG = LogFactory.getLog(SparkDynamicPartitionPruner.class); + private final Map> sourceInfoMap = new LinkedHashMap>(); + private final BytesWritable writable = new BytesWritable(); + + public void prune(MapWork work, JobConf jobConf) throws HiveException, SerDeException { + sourceInfoMap.clear(); + initialize(work, jobConf); + if (sourceInfoMap.size() == 0) { + // Nothing to prune for this MapWork + return; + } + processFiles(work, jobConf); + prunePartitions(work); + } + + public void initialize(MapWork work, JobConf jobConf) throws SerDeException { + Map columnMap = new HashMap(); + Set sourceWorkIds = work.getEventSourceTableDescMap().keySet(); + + for (String id : sourceWorkIds) { + List tables = work.getEventSourceTableDescMap().get(id); + List columnNames = work.getEventSourceColumnNameMap().get(id); + List partKeyExprs = work.getEventSourcePartKeyExprMap().get(id); + + Iterator cit = columnNames.iterator(); + Iterator pit = partKeyExprs.iterator(); + for (TableDesc t : tables) { + String columnName = cit.next(); + ExprNodeDesc partKeyExpr = pit.next(); + SourceInfo si = new SourceInfo(t, partKeyExpr, columnName, jobConf); + if (!sourceInfoMap.containsKey(id)) { + sourceInfoMap.put(id, new ArrayList()); + } + sourceInfoMap.get(id).add(si); + + // We could have multiple sources restrict the same column, need to take + // the union of the values in that case. + if (columnMap.containsKey(columnName)) { + si.values = columnMap.get(columnName).values; + } + columnMap.put(columnName, si); + } + } + } + + private void processFiles(MapWork work, JobConf jobConf) throws HiveException { + ObjectInputStream in = null; + try { + Path baseDir = work.getTmpPathForPartitionPruning(); + FileSystem fs = FileSystem.get(baseDir.toUri(), jobConf); + + // Find the SourceInfo to put values in. + for (String name : sourceInfoMap.keySet()) { + Path sourceDir = new Path(baseDir, name); + for (FileStatus fstatus : fs.listStatus(sourceDir)) { + LOG.info("Start processing pruning file: " + fstatus.getPath()); + in = new ObjectInputStream(fs.open(fstatus.getPath())); + String columnName = in.readUTF(); + SourceInfo info = null; + + for (SourceInfo si : sourceInfoMap.get(name)) { + if (columnName.equals(si.columnName)) { + info = si; + break; + } + } + + Preconditions.checkArgument(info != null, + "AssertionError: no source info for the column: " + columnName); + + // Read fields + while (in.available() > 0) { + writable.readFields(in); + + Object row = info.deserializer.deserialize(writable); + Object value = info.soi.getStructFieldData(row, info.field); + value = ObjectInspectorUtils.copyToStandardObject(value, info.fieldInspector); + info.values.add(value); + } + } + } + } catch (Exception e) { + throw new HiveException(e); + } finally { + try { + if (in != null) { + in.close(); + } + } catch (IOException e) { + throw new HiveException("error while trying to close input stream", e); + } + } + } + + private void prunePartitions(MapWork work) throws HiveException { + for (String source : sourceInfoMap.keySet()) { + for (SourceInfo info : sourceInfoMap.get(source)) { + prunePartitionSingleSource(info, work); + } + } + } + + private void prunePartitionSingleSource(SourceInfo info, MapWork work) + throws HiveException { + Set values = info.values; + String columnName = info.columnName; + + ObjectInspector oi = + PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(TypeInfoFactory + .getPrimitiveTypeInfo(info.fieldInspector.getTypeName())); + + ObjectInspectorConverters.Converter converter = + ObjectInspectorConverters.getConverter( + PrimitiveObjectInspectorFactory.javaStringObjectInspector, oi); + + StructObjectInspector soi = + ObjectInspectorFactory.getStandardStructObjectInspector( + Collections.singletonList(columnName), Collections.singletonList(oi)); + + @SuppressWarnings("rawtypes") + ExprNodeEvaluator eval = ExprNodeEvaluatorFactory.get(info.partKey); + eval.initialize(soi); + + applyFilterToPartitions(work, converter, eval, columnName, values); + } + + private void applyFilterToPartitions( + MapWork work, + ObjectInspectorConverters.Converter converter, + ExprNodeEvaluator eval, + String columnName, + Set values) throws HiveException { + + Object[] row = new Object[1]; + + Iterator it = work.getPathToPartitionInfo().keySet().iterator(); + while (it.hasNext()) { + String p = it.next(); + PartitionDesc desc = work.getPathToPartitionInfo().get(p); + Map spec = desc.getPartSpec(); + if (spec == null) { + throw new AssertionException("No partition spec found in dynamic pruning"); + } + + String partValueString = spec.get(columnName); + if (partValueString == null) { + throw new AssertionException("Could not find partition value for column: " + columnName); + } + + Object partValue = converter.convert(partValueString); + if (LOG.isDebugEnabled()) { + LOG.debug("Converted partition value: " + partValue + " original (" + partValueString + ")"); + } + + row[0] = partValue; + partValue = eval.evaluate(row); + if (LOG.isDebugEnabled()) { + LOG.debug("part key expr applied: " + partValue); + } + + if (!values.contains(partValue)) { + LOG.info("Pruning path: " + p); + it.remove(); + work.getPathToAliases().remove(p); + work.getPaths().remove(p); + work.getPartitionDescs().remove(desc); + } + } + } + + @SuppressWarnings("deprecation") + private static class SourceInfo { + final ExprNodeDesc partKey; + final Deserializer deserializer; + final StructObjectInspector soi; + final StructField field; + final ObjectInspector fieldInspector; + Set values = new HashSet(); + final String columnName; + + SourceInfo(TableDesc table, ExprNodeDesc partKey, String columnName, JobConf jobConf) + throws SerDeException { + this.partKey = partKey; + this.columnName = columnName; + + deserializer = ReflectionUtils.newInstance(table.getDeserializerClass(), null); + deserializer.initialize(jobConf, table.getProperties()); + + ObjectInspector inspector = deserializer.getObjectInspector(); + if (LOG.isDebugEnabled()) { + LOG.debug("Type of obj insp: " + inspector.getTypeName()); + } + + soi = (StructObjectInspector) inspector; + List fields = soi.getAllStructFieldRefs(); + assert(fields.size() > 1) : "expecting single field in input"; + + field = fields.get(0); + fieldInspector = + ObjectInspectorUtils.getStandardObjectInspector(field.getFieldObjectInspector()); + } + } + +} 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 ca0ffb6..cf2c3bc 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 @@ -23,16 +23,22 @@ import java.net.URISyntaxException; import java.util.Collection; import java.util.UUID; +import java.util.Collection; +import com.google.common.base.Preconditions; import org.apache.commons.io.FilenameUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.hive.ql.exec.TaskFactory; 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.io.HiveKey; import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.plan.BaseWork; +import org.apache.hadoop.hive.ql.plan.SparkWork; import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hadoop.io.BytesWritable; import org.apache.spark.Dependency; @@ -158,4 +164,54 @@ private static void rddToString(RDD rdd, StringBuilder sb, String offset) { } } } + + /** + * Generate a temporary path for dynamic partition pruning in Spark branch + * TODO: no longer need this if we use accumulator! + * @param basePath + * @param id + * @return + */ + public static Path generateTmpPathForPartitionPruning(Path basePath, String id) { + return new Path(basePath, id); + } + + /** + * Return the ID for this BaseWork, in String form. + * @param work the input BaseWork + * @return the unique ID for this BaseWork + */ + public static String getWorkId(BaseWork work) { + String workName = work.getName(); + return workName.substring(workName.indexOf(" ") + 1); + } + + public static SparkTask createSparkTask(HiveConf conf) { + return (SparkTask) TaskFactory.get( + new SparkWork(conf.getVar(HiveConf.ConfVars.HIVEQUERYID)), conf); + } + + public static SparkTask createSparkTask(SparkWork work, HiveConf conf) { + return (SparkTask) TaskFactory.get(work, conf); + } + + /** + * Recursively find all operators under root, that are of class clazz, and + * put them in result. + * @param result all operators under root that are of class clazz + * @param root the root operator under which all operators will be examined + * @param clazz clas to collect. Must NOT be null. + */ + public static void collectOp(Collection> result, Operator root, Class clazz) { + Preconditions.checkArgument(clazz != null, "AssertionError: clazz should not be null"); + if (root == null) { + return; + } + if (clazz.equals(root.getClass())) { + result.add(root); + } + for (Operator child : root.getChildOperators()) { + collectOp(result, child, clazz); + } + } } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSparkPartitionPruningSinkOperator.java ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSparkPartitionPruningSinkOperator.java new file mode 100644 index 0000000..3bce49d --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSparkPartitionPruningSinkOperator.java @@ -0,0 +1,99 @@ +/** + * 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.vector; + +import java.util.Collection; +import java.util.concurrent.Future; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.optimizer.spark.SparkPartitionPruningSinkDesc; +import org.apache.hadoop.hive.ql.parse.spark.SparkPartitionPruningSinkOperator; +import org.apache.hadoop.hive.ql.plan.OperatorDesc; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.io.Writable; + +/** + * Vectorized version for SparkPartitionPruningSinkOperator. + * Forked from VectorAppMasterEventOperator. + **/ +public class VectorSparkPartitionPruningSinkOperator extends SparkPartitionPruningSinkOperator { + + private static final long serialVersionUID = 1L; + + private VectorizationContext vContext; + + protected transient boolean firstBatch; + + protected transient VectorExtractRowDynBatch vectorExtractRowDynBatch; + + protected transient Object[] singleRow; + + public VectorSparkPartitionPruningSinkOperator(VectorizationContext context, + OperatorDesc conf) { + super(); + this.conf = (SparkPartitionPruningSinkDesc) conf; + this.vContext = context; + } + + public VectorSparkPartitionPruningSinkOperator() { + } + + @Override + public Collection> initializeOp(Configuration hconf) throws HiveException { + inputObjInspectors[0] = + VectorizedBatchUtil.convertToStandardStructObjectInspector( + (StructObjectInspector) inputObjInspectors[0]); + Collection> result = super.initializeOp(hconf); + assert result.isEmpty(); + + firstBatch = true; + + return result; + } + + @Override + public void process(Object data, int tag) throws HiveException { + VectorizedRowBatch batch = (VectorizedRowBatch) data; + if (firstBatch) { + vectorExtractRowDynBatch = new VectorExtractRowDynBatch(); + vectorExtractRowDynBatch.init((StructObjectInspector) inputObjInspectors[0], + vContext.getProjectedColumns()); + singleRow = new Object[vectorExtractRowDynBatch.getCount()]; + firstBatch = false; + } + + vectorExtractRowDynBatch.setBatchOnEntry(batch); + ObjectInspector rowInspector = inputObjInspectors[0]; + try { + Writable writableRow; + for (int logical = 0; logical < batch.size; logical++) { + int batchIndex = batch.selectedInUse ? batch.selected[logical] : logical; + vectorExtractRowDynBatch.extractRow(batchIndex, singleRow); + writableRow = serializer.serialize(singleRow, rowInspector); + writableRow.write(buffer); + } + } catch (Exception e) { + throw new HiveException(e); + } + + vectorExtractRowDynBatch.forgetBatchOnExit(); + } +} diff --git ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java index 1de7e40..e13c4dd 100644 --- ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java +++ ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveInputFormat.java @@ -25,6 +25,7 @@ import java.util.Arrays; import java.util.HashMap; import java.util.HashSet; +import java.util.Iterator; import java.util.LinkedList; import java.util.List; import java.util.Map; @@ -514,34 +515,40 @@ public int hashCode() { int numThreads = Math.min(MAX_CHECK_NONCOMBINABLE_THREAD_NUM, (int) Math.ceil((double) paths.length / DEFAULT_NUM_PATH_PER_THREAD)); int numPathPerThread = (int) Math.ceil((double) paths.length / numThreads); - LOG.info("Total number of paths: " + paths.length + - ", launching " + numThreads + " threads to check non-combinable ones."); - ExecutorService executor = Executors.newFixedThreadPool(numThreads); - List>> futureList = new ArrayList>>(numThreads); - try { - for (int i = 0; i < numThreads; i++) { - int start = i * numPathPerThread; - int length = i != numThreads - 1 ? numPathPerThread : paths.length - start; - futureList.add(executor.submit( - new CheckNonCombinablePathCallable(paths, start, length, job))); - } - Set nonCombinablePathIndices = new HashSet(); - for (Future> future : futureList) { - nonCombinablePathIndices.addAll(future.get()); - } - for (int i = 0; i < paths.length; i++) { - if (nonCombinablePathIndices.contains(i)) { - nonCombinablePaths.add(paths[i]); - } else { - combinablePaths.add(paths[i]); + + // This check is necessary because for Spark branch, the result array from + // getInputPaths() above could be empty, and therefore numThreads could be 0. + // In that case, Executors.newFixedThreadPool will fail. + if (numThreads > 0) { + LOG.info("Total number of paths: " + paths.length + + ", launching " + numThreads + " threads to check non-combinable ones."); + ExecutorService executor = Executors.newFixedThreadPool(numThreads); + List>> futureList = new ArrayList>>(numThreads); + try { + for (int i = 0; i < numThreads; i++) { + int start = i * numPathPerThread; + int length = i != numThreads - 1 ? numPathPerThread : paths.length - start; + futureList.add(executor.submit( + new CheckNonCombinablePathCallable(paths, start, length, job))); + } + Set nonCombinablePathIndices = new HashSet(); + for (Future> future : futureList) { + nonCombinablePathIndices.addAll(future.get()); + } + for (int i = 0; i < paths.length; i++) { + if (nonCombinablePathIndices.contains(i)) { + nonCombinablePaths.add(paths[i]); + } else { + combinablePaths.add(paths[i]); + } } + } catch (Exception e) { + LOG.error("Error checking non-combinable path", e); + perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.GET_SPLITS); + throw new IOException(e); + } finally { + executor.shutdownNow(); } - } catch (Exception e) { - LOG.error("Error checking non-combinable path", e); - perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.GET_SPLITS); - throw new IOException(e); - } finally { - executor.shutdownNow(); } // Store the previous value for the path specification diff --git ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java index 2ff3951..fd16b35 100755 --- ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java +++ ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java @@ -26,6 +26,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.Map.Entry; @@ -37,6 +38,7 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.io.HiveIOExceptionHandlerUtil; import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; +import org.apache.hadoop.hive.ql.exec.spark.SparkDynamicPartitionPruner; import org.apache.hadoop.hive.ql.plan.TableDesc; import org.apache.hadoop.hive.ql.exec.Operator; import org.apache.hadoop.hive.ql.exec.TableScanOperator; @@ -266,6 +268,18 @@ protected void init(JobConf job) { } else { mrwork = Utilities.getMapWork(job); } + + // Prune partitions + if (HiveConf.getVar(job, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE).equals("spark") + && HiveConf.getBoolVar(job, HiveConf.ConfVars.SPARK_DYNAMIC_PARTITION_PRUNING)) { + SparkDynamicPartitionPruner pruner = new SparkDynamicPartitionPruner(); + try { + pruner.prune(mrwork, job); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + pathToPartitionInfo = mrwork.getPathToPartitionInfo(); } } @@ -309,18 +323,28 @@ private void addSplitsForGroup(List dirs, TableScanOperator tableScan, Job } Path[] getInputPaths(JobConf job) throws IOException { - Path[] dirs = FileInputFormat.getInputPaths(job); - if (dirs.length == 0) { - // on tez we're avoiding to duplicate the file info in FileInputFormat. - if (HiveConf.getVar(job, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE).equals("tez")) { - try { - List paths = Utilities.getInputPathsTez(job, mrwork); - dirs = paths.toArray(new Path[paths.size()]); - } catch (Exception e) { - throw new IOException("Could not create input files", e); + Path[] dirs; + if (HiveConf.getVar(job, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE).equals("spark")) { + Set pathStrings = mrwork.getPathToPartitionInfo().keySet(); + dirs = new Path[pathStrings.size()]; + Iterator it = pathStrings.iterator(); + for (int i = 0; i < dirs.length; i++) { + dirs[i] = new Path(it.next()); + } + } else { + dirs = FileInputFormat.getInputPaths(job); + if (dirs.length == 0) { + // on tez we're avoiding to duplicate the file info in FileInputFormat. + if (HiveConf.getVar(job, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE).equals("tez")) { + try { + List paths = Utilities.getInputPathsTez(job, mrwork); + dirs = paths.toArray(new Path[paths.size()]); + } catch (Exception e) { + throw new IOException("Could not create input files", e); + } + } else { + throw new IOException("No input paths specified in job"); } - } else { - throw new IOException("No input paths specified in job"); } } return dirs; diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java index 8546d21..f475926 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java @@ -48,10 +48,12 @@ import org.apache.hadoop.hive.ql.lib.RuleRegExp; import org.apache.hadoop.hive.ql.metadata.Partition; import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.ql.optimizer.spark.SparkPartitionPruningSinkDesc; import org.apache.hadoop.hive.ql.parse.OptimizeTezProcContext; import org.apache.hadoop.hive.ql.parse.ParseContext; import org.apache.hadoop.hive.ql.parse.PrunedPartitionList; import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.parse.spark.OptimizeSparkProcContext; import org.apache.hadoop.hive.ql.plan.AggregationDesc; import org.apache.hadoop.hive.ql.plan.DynamicPruningEventDesc; import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc; @@ -152,15 +154,24 @@ private String extractColName(ExprNodeDesc root) { @Override public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, Object... nodeOutputs) throws SemanticException { - OptimizeTezProcContext context = (OptimizeTezProcContext) procCtx; - ParseContext parseContext = context.parseContext; + ParseContext parseContext; + if (procCtx instanceof OptimizeTezProcContext) { + parseContext = ((OptimizeTezProcContext) procCtx).parseContext; + } else if (procCtx instanceof OptimizeSparkProcContext) { + parseContext = ((OptimizeSparkProcContext) procCtx).getParseContext(); + } else { + throw new IllegalArgumentException("expected parseContext to be either " + + "OptimizeTezProcContext or OptimizeSparkProcContext, but found " + + procCtx.getClass().getName()); + } FilterOperator filter = (FilterOperator) nd; FilterDesc desc = filter.getConf(); TableScanOperator ts = null; - if (!parseContext.getConf().getBoolVar(ConfVars.TEZ_DYNAMIC_PARTITION_PRUNING)) { + if (!parseContext.getConf().getBoolVar(ConfVars.TEZ_DYNAMIC_PARTITION_PRUNING) && + !parseContext.getConf().getBoolVar(ConfVars.SPARK_DYNAMIC_PARTITION_PRUNING)) { // nothing to do when the optimization is off return null; } @@ -311,14 +322,25 @@ private void generateEventOperatorPlan(DynamicListContext ctx, ParseContext pars groupByOp.setColumnExprMap(colMap); // finally add the event broadcast operator - DynamicPruningEventDesc eventDesc = new DynamicPruningEventDesc(); - eventDesc.setTableScan(ts); - eventDesc.setTable(PlanUtils.getReduceValueTableDesc(PlanUtils - .getFieldSchemasFromColumnList(keyExprs, "key"))); - eventDesc.setTargetColumnName(column); - eventDesc.setPartKey(partKey); - - OperatorFactory.getAndMakeChild(eventDesc, groupByOp); + if (HiveConf.getVar(parseContext.getConf(), + ConfVars.HIVE_EXECUTION_ENGINE).equals("tez")) { + DynamicPruningEventDesc eventDesc = new DynamicPruningEventDesc(); + eventDesc.setTableScan(ts); + eventDesc.setTable(PlanUtils.getReduceValueTableDesc(PlanUtils + .getFieldSchemasFromColumnList(keyExprs, "key"))); + eventDesc.setTargetColumnName(column); + eventDesc.setPartKey(partKey); + OperatorFactory.getAndMakeChild(eventDesc, groupByOp); + } else { + // Must be spark branch + SparkPartitionPruningSinkDesc desc = new SparkPartitionPruningSinkDesc(); + desc.setTableScan(ts); + desc.setTable(PlanUtils.getReduceValueTableDesc(PlanUtils + .getFieldSchemasFromColumnList(keyExprs, "key"))); + desc.setTargetColumnName(column); + desc.setPartKey(partKey); + OperatorFactory.getAndMakeChild(desc, groupByOp); + } } private Map walkExprTree(ExprNodeDesc pred, NodeProcessorCtx ctx) diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java index a7cf8b7..e6db133 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java @@ -167,7 +167,7 @@ public void initialize(HiveConf hiveConf) { if(HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTIMIZEMETADATAQUERIES)) { transformations.add(new StatsOptimizer()); } - if (isSparkExecEngine || (pctx.getContext().getExplain() && !isTezExecEngine)) { + if (pctx.getContext().getExplain() && !isTezExecEngine && !isSparkExecEngine) { transformations.add(new AnnotateWithStatistics()); transformations.add(new AnnotateWithOpTraits()); } diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/SparkRemoveDynamicPruningBySize.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/SparkRemoveDynamicPruningBySize.java new file mode 100644 index 0000000..3742857 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/SparkRemoveDynamicPruningBySize.java @@ -0,0 +1,73 @@ +/** + * 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.optimizer; + +import java.util.Stack; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.hive.ql.lib.Node; +import org.apache.hadoop.hive.ql.lib.NodeProcessor; +import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; +import org.apache.hadoop.hive.ql.optimizer.spark.SparkPartitionPruningSinkDesc; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.parse.spark.OptimizeSparkProcContext; +import org.apache.hadoop.hive.ql.parse.spark.SparkPartitionPruningSinkOperator; + +/** + * If we expect the number of keys for dynamic pruning to be too large we + * disable it. + * + * Cloned from RemoveDynamicPruningBySize + */ +public class SparkRemoveDynamicPruningBySize implements NodeProcessor { + + static final private Log LOG = LogFactory.getLog(SparkRemoveDynamicPruningBySize.class.getName()); + + @Override + public Object process(Node nd, Stack stack, NodeProcessorCtx procContext, + Object... nodeOutputs) + throws SemanticException { + + OptimizeSparkProcContext context = (OptimizeSparkProcContext) procContext; + + SparkPartitionPruningSinkOperator op = (SparkPartitionPruningSinkOperator) nd; + SparkPartitionPruningSinkDesc desc = op.getConf(); + + if (desc.getStatistics().getDataSize() > context.getConf() + .getLongVar(ConfVars.SPARK_DYNAMIC_PARTITION_PRUNING_MAX_DATA_SIZE)) { + Operator child = op; + Operator curr = op; + + while (curr.getChildOperators().size() <= 1) { + child = curr; + curr = curr.getParentOperators().get(0); + } + + curr.removeChild(child); + // at this point we've found the fork in the op pipeline that has the pruning as a child plan. + LOG.info("Disabling dynamic pruning for: " + + desc.getTableScan().getName() + + ". Expected data size is too big: " + desc.getStatistics().getDataSize()); + } + return false; + } +} \ No newline at end of file diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java index ad47547..180513e 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java @@ -919,6 +919,7 @@ boolean validateMapWorkOperator(Operator op, MapWork mWo case FILESINK: case LIMIT: case EVENT: + case SPARKPRUNINGSINK: ret = true; break; case HASHTABLESINK: @@ -965,6 +966,7 @@ boolean validateReduceWorkOperator(Operator op) { break; case LIMIT: case EVENT: + case SPARKPRUNINGSINK: ret = true; break; case HASHTABLESINK: diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkPartitionPruningSinkDesc.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkPartitionPruningSinkDesc.java new file mode 100644 index 0000000..cc78227 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkPartitionPruningSinkDesc.java @@ -0,0 +1,100 @@ +/** + * 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.optimizer.spark; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.ql.exec.TableScanOperator; +import org.apache.hadoop.hive.ql.plan.AbstractOperatorDesc; +import org.apache.hadoop.hive.ql.plan.Explain; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.TableDesc; + +@Explain(displayName = "Spark Partition Pruning Sink Operator") +public class SparkPartitionPruningSinkDesc extends AbstractOperatorDesc { + + // column in the target table that will be pruned against + private String targetColumnName; + + private TableDesc table; + + private transient TableScanOperator tableScan; + + // the partition column we're interested in + private ExprNodeDesc partKey; + + private Path path; + + private String targetWork; + + @Explain(displayName = "tmp Path", explainLevels = { Explain.Level.EXTENDED }) + public Path getPath() { + return path; + } + + public void setPath(Path path) { + this.path = path; + } + + @Explain(displayName = "target work") + public String getTargetWork() { + return this.targetWork; + } + + public void setTargetWork(String targetWork) { + this.targetWork = targetWork; + } + + public TableScanOperator getTableScan() { + return tableScan; + } + + public void setTableScan(TableScanOperator tableScan) { + this.tableScan = tableScan; + } + + @Explain(displayName = "target column name") + public String getTargetColumnName() { + return targetColumnName; + } + + public void setTargetColumnName(String targetColumnName) { + this.targetColumnName = targetColumnName; + } + + public ExprNodeDesc getPartKey() { + return partKey; + } + + public void setPartKey(ExprNodeDesc partKey) { + this.partKey = partKey; + } + + public TableDesc getTable() { + return table; + } + + public void setTable(TableDesc table) { + this.table = table; + } + + @Explain(displayName = "partition key expr") + public String getPartKeyString() { + return partKey.getExprString(); + } +} diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java index 447f104..0a0c791 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java @@ -30,6 +30,7 @@ import org.apache.hadoop.hive.ql.exec.TaskFactory; import org.apache.hadoop.hive.ql.exec.UnionOperator; import org.apache.hadoop.hive.ql.exec.spark.SparkTask; +import org.apache.hadoop.hive.ql.exec.spark.SparkUtilities; import org.apache.hadoop.hive.ql.hooks.ReadEntity; import org.apache.hadoop.hive.ql.hooks.WriteEntity; import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; @@ -40,7 +41,6 @@ import org.apache.hadoop.hive.ql.plan.OperatorDesc; import org.apache.hadoop.hive.ql.plan.ReduceWork; import org.apache.hadoop.hive.ql.plan.SparkEdgeProperty; -import org.apache.hadoop.hive.ql.plan.SparkWork; import java.io.Serializable; import java.util.HashMap; @@ -138,6 +138,13 @@ // This is necessary as sometimes semantic analyzer's mapping is different than operator's own alias. public final Map> topOps; + // The set of pruning sinks + public final Set> pruningSinkSet; + + // The set of TableScanOperators for pruning OP trees + public final Set> clonedPruningTableScanSet; + + @SuppressWarnings("unchecked") public GenSparkProcContext(HiveConf conf, ParseContext parseContext, @@ -153,8 +160,7 @@ public GenSparkProcContext(HiveConf conf, this.inputs = inputs; this.outputs = outputs; this.topOps = topOps; - this.currentTask = (SparkTask) TaskFactory.get( - new SparkWork(conf.getVar(HiveConf.ConfVars.HIVEQUERYID)), conf); + this.currentTask = SparkUtilities.createSparkTask(conf); this.rootTasks.add(currentTask); this.leafOpToFollowingWorkInfo = new LinkedHashMap>(); @@ -177,5 +183,7 @@ public GenSparkProcContext(HiveConf conf, this.clonedReduceSinks = new LinkedHashSet(); this.fileSinkSet = new LinkedHashSet(); this.fileSinkMap = new LinkedHashMap>(); + this.pruningSinkSet = new LinkedHashSet>(); + this.clonedPruningTableScanSet = new LinkedHashSet>(); } } diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java index 7992c88..1c0b79d 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java @@ -46,7 +46,9 @@ import org.apache.hadoop.hive.ql.exec.TableScanOperator; import org.apache.hadoop.hive.ql.exec.UnionOperator; import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.exec.spark.SparkUtilities; import org.apache.hadoop.hive.ql.optimizer.GenMapRedUtils; +import org.apache.hadoop.hive.ql.optimizer.spark.SparkPartitionPruningSinkDesc; import org.apache.hadoop.hive.ql.optimizer.spark.SparkSortMergeJoinFactory; import org.apache.hadoop.hive.ql.parse.ParseContext; import org.apache.hadoop.hive.ql.parse.PrunedPartitionList; @@ -61,6 +63,7 @@ import com.google.common.base.Preconditions; import com.google.common.base.Strings; +import org.apache.hadoop.hive.ql.plan.TableDesc; /** * GenSparkUtils is a collection of shared helper methods to produce SparkWork @@ -132,7 +135,7 @@ protected void setupReduceSink(GenSparkProcContext context, ReduceWork reduceWor // remember which parent belongs to which tag reduceWork.getTagToInput().put(reduceSink.getConf().getTag(), - context.preceedingWork.getName()); + context.preceedingWork.getName()); // remember the output name of the reduce sink reduceSink.getConf().setOutputName(reduceWork.getName()); @@ -218,6 +221,16 @@ public void removeUnionOperators(Configuration conf, GenSparkProcContext context Iterator> newOpQueueIt = newOpQueue.iterator(); for (Operator op : opQueue) { Operator newOp = newOpQueueIt.next(); + + // We need to update rootToWorkMap in case the op is a key, since even + // though we clone the op tree, we're still using the same MapWork/ReduceWork. + if (context.rootToWorkMap.containsKey(op)) { + context.rootToWorkMap.put(newOp, context.rootToWorkMap.get(op)); + } + // Don't remove the old entry - in SparkPartitionPruningSink it still + // refers to the old TS, and we need to lookup it later in + // processPartitionPruningSink. + if (op instanceof FileSinkOperator) { List fileSinkList = context.fileSinkMap.get(op); if (fileSinkList == null) { @@ -225,6 +238,12 @@ public void removeUnionOperators(Configuration conf, GenSparkProcContext context } fileSinkList.add((FileSinkOperator) newOp); context.fileSinkMap.put((FileSinkOperator) op, fileSinkList); + } else if (op instanceof SparkPartitionPruningSinkOperator) { + SparkPartitionPruningSinkOperator oldPruningSink = (SparkPartitionPruningSinkOperator) op; + SparkPartitionPruningSinkOperator newPruningSink = (SparkPartitionPruningSinkOperator) newOp; + newPruningSink.getConf().setTableScan(oldPruningSink.getConf().getTableScan()); + context.pruningSinkSet.add(newPruningSink); + context.pruningSinkSet.remove(oldPruningSink); } } } @@ -337,6 +356,67 @@ public void processFileSink(GenSparkProcContext context, FileSinkOperator fileSi } } + /** + * Populate partition pruning information from the pruning sink operator to the + * target MapWork (the MapWork for the big table side). The information include the source table + * name, column name, and partition key expression. It also set up the temporary path used to + * communicate between the target MapWork and source BaseWork. + * + * Here "source" refers to the small table side, while "target" refers to the big + * table side. + * + * @param context the spark context. + * @param pruningSink the pruner sink operator being processed. + */ + public void processPartitionPruningSink(GenSparkProcContext context, + SparkPartitionPruningSinkOperator pruningSink) { + SparkPartitionPruningSinkDesc desc = pruningSink.getConf(); + TableScanOperator ts = desc.getTableScan(); + MapWork targetWork = (MapWork) context.rootToWorkMap.get(ts); + + Preconditions.checkArgument( + targetWork != null, + "No targetWork found for tablescan " + ts); + + String targetId = SparkUtilities.getWorkId(targetWork); + + BaseWork sourceWork = getEnclosingWork(pruningSink, context); + String sourceId = SparkUtilities.getWorkId(sourceWork); + + // set up temporary path to communicate between the small/big table + Path tmpPath = targetWork.getTmpPathForPartitionPruning(); + if (tmpPath == null) { + Path baseTmpPath = context.parseContext.getContext().getMRTmpPath(); + tmpPath = SparkUtilities.generateTmpPathForPartitionPruning(baseTmpPath, targetId); + targetWork.setTmpPathForPartitionPruning(tmpPath); + LOG.info("Setting tmp path between source work and target work:\n" + tmpPath); + } + + desc.setPath(new Path(tmpPath, sourceId)); + desc.setTargetWork(targetWork.getName()); + + // store table descriptor in map-targetWork + if (!targetWork.getEventSourceTableDescMap().containsKey(sourceId)) { + targetWork.getEventSourceTableDescMap().put(sourceId, new LinkedList()); + } + List tables = targetWork.getEventSourceTableDescMap().get(sourceId); + tables.add(pruningSink.getConf().getTable()); + + // store column name in map-targetWork + if (!targetWork.getEventSourceColumnNameMap().containsKey(sourceId)) { + targetWork.getEventSourceColumnNameMap().put(sourceId, new LinkedList()); + } + List columns = targetWork.getEventSourceColumnNameMap().get(sourceId); + columns.add(desc.getTargetColumnName()); + + // store partition key expr in map-targetWork + if (!targetWork.getEventSourcePartKeyExprMap().containsKey(sourceId)) { + targetWork.getEventSourcePartKeyExprMap().put(sourceId, new LinkedList()); + } + List keys = targetWork.getEventSourcePartKeyExprMap().get(sourceId); + keys.add(desc.getPartKey()); + } + public static SparkEdgeProperty getEdgeProperty(ReduceSinkOperator reduceSink, ReduceWork reduceWork) throws SemanticException { SparkEdgeProperty edgeProperty = new SparkEdgeProperty(SparkEdgeProperty.SHUFFLE_NONE); @@ -490,4 +570,33 @@ private static boolean hasGBYOperator(ReduceSinkOperator rs) { } return false; } + + /** + * getEncosingWork finds the BaseWork any given operator belongs to. + */ + public BaseWork getEnclosingWork(Operator op, GenSparkProcContext procCtx) { + List> ops = new ArrayList>(); + findRoots(op, ops); + for (Operator r : ops) { + BaseWork work = procCtx.rootToWorkMap.get(r); + if (work != null) { + return work; + } + } + return null; + } + + /* + * findRoots returns all root operators (in ops) that result in operator op + */ + private void findRoots(Operator op, List> ops) { + List> parents = op.getParentOperators(); + if (parents == null || parents.isEmpty()) { + ops.add(op); + return; + } + for (Operator p : parents) { + findRoots(p, ops); + } + } } diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/spark/OptimizeSparkProcContext.java ql/src/java/org/apache/hadoop/hive/ql/parse/spark/OptimizeSparkProcContext.java index f7586a4..3b71af1 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/spark/OptimizeSparkProcContext.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/spark/OptimizeSparkProcContext.java @@ -20,15 +20,12 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.exec.MapJoinOperator; -import org.apache.hadoop.hive.ql.exec.Operator; import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; import org.apache.hadoop.hive.ql.hooks.ReadEntity; import org.apache.hadoop.hive.ql.hooks.WriteEntity; import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; import org.apache.hadoop.hive.ql.parse.ParseContext; -import org.apache.hadoop.hive.ql.plan.OperatorDesc; -import java.util.Deque; import java.util.HashMap; import java.util.HashSet; import java.util.Map; @@ -49,19 +46,12 @@ private final Set visitedReduceSinks = new HashSet(); private final Map mjOpSizes = new HashMap(); - // rootOperators are all the table scan operators in sequence - // of traversal - private final Deque> rootOperators; - public OptimizeSparkProcContext(HiveConf conf, ParseContext parseContext, - Set inputs, Set outputs, - Deque> rootOperators) { - + Set inputs, Set outputs) { this.conf = conf; this.parseContext = parseContext; this.inputs = inputs; this.outputs = outputs; - this.rootOperators = rootOperators; } public ParseContext getParseContext() { @@ -84,10 +74,6 @@ public HiveConf getConf() { return visitedReduceSinks; } - public Deque> getRootOperators() { - return rootOperators; - } - public Map getMjOpSizes() { return mjOpSizes; } diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java index 7f2c079..27a1d99 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java @@ -19,10 +19,8 @@ import java.io.Serializable; import java.util.ArrayList; -import java.util.Deque; import java.util.HashMap; import java.util.LinkedHashMap; -import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Set; @@ -35,6 +33,7 @@ import org.apache.hadoop.hive.ql.exec.ConditionalTask; import org.apache.hadoop.hive.ql.exec.DummyStoreOperator; import org.apache.hadoop.hive.ql.exec.FileSinkOperator; +import org.apache.hadoop.hive.ql.exec.FilterOperator; import org.apache.hadoop.hive.ql.exec.JoinOperator; import org.apache.hadoop.hive.ql.exec.MapJoinOperator; import org.apache.hadoop.hive.ql.exec.Operator; @@ -44,12 +43,14 @@ import org.apache.hadoop.hive.ql.exec.Task; import org.apache.hadoop.hive.ql.exec.UnionOperator; import org.apache.hadoop.hive.ql.exec.spark.SparkTask; +import org.apache.hadoop.hive.ql.exec.spark.SparkUtilities; import org.apache.hadoop.hive.ql.hooks.ReadEntity; import org.apache.hadoop.hive.ql.hooks.WriteEntity; import org.apache.hadoop.hive.ql.lib.CompositeProcessor; import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker; import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher; import org.apache.hadoop.hive.ql.lib.Dispatcher; +import org.apache.hadoop.hive.ql.lib.ForwardWalker; import org.apache.hadoop.hive.ql.lib.GraphWalker; import org.apache.hadoop.hive.ql.lib.Node; import org.apache.hadoop.hive.ql.lib.NodeProcessor; @@ -58,7 +59,10 @@ import org.apache.hadoop.hive.ql.lib.RuleRegExp; import org.apache.hadoop.hive.ql.lib.TypeRule; import org.apache.hadoop.hive.ql.log.PerfLogger; -import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.hive.ql.optimizer.ConstantPropagate; +import org.apache.hadoop.hive.ql.optimizer.DynamicPartitionPruningOptimization; +import org.apache.hadoop.hive.ql.optimizer.SparkRemoveDynamicPruningBySize; +import org.apache.hadoop.hive.ql.optimizer.metainfo.annotation.AnnotateWithOpTraits; import org.apache.hadoop.hive.ql.optimizer.physical.MetadataOnlyOptimizer; import org.apache.hadoop.hive.ql.optimizer.physical.NullScanOptimizer; import org.apache.hadoop.hive.ql.optimizer.physical.PhysicalContext; @@ -72,8 +76,8 @@ import org.apache.hadoop.hive.ql.optimizer.spark.SparkJoinOptimizer; import org.apache.hadoop.hive.ql.optimizer.spark.SparkReduceSinkMapJoinProc; import org.apache.hadoop.hive.ql.optimizer.spark.SparkSkewJoinResolver; -import org.apache.hadoop.hive.ql.optimizer.spark.SparkSortMergeJoinFactory; import org.apache.hadoop.hive.ql.optimizer.spark.SplitSparkWorkResolver; +import org.apache.hadoop.hive.ql.optimizer.stats.annotation.AnnotateWithStatistics; import org.apache.hadoop.hive.ql.parse.GlobalLimitCtx; import org.apache.hadoop.hive.ql.parse.ParseContext; import org.apache.hadoop.hive.ql.parse.SemanticException; @@ -83,7 +87,6 @@ import org.apache.hadoop.hive.ql.plan.MoveWork; import org.apache.hadoop.hive.ql.plan.OperatorDesc; import org.apache.hadoop.hive.ql.plan.SparkWork; -import org.apache.hadoop.hive.ql.session.SessionState.LogHelper; /** * SparkCompiler translates the operator plan into SparkTasks. @@ -102,22 +105,70 @@ public SparkCompiler() { protected void optimizeOperatorPlan(ParseContext pCtx, Set inputs, Set outputs) throws SemanticException { PERF_LOGGER.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_OPTIMIZE_OPERATOR_TREE); - // Sequence of TableScan operators to be walked - Deque> deque = new LinkedList>(); - deque.addAll(pCtx.getTopOps().values()); - OptimizeSparkProcContext procCtx = new OptimizeSparkProcContext(conf, pCtx, inputs, outputs, deque); - // create a walker which walks the tree in a DFS manner while maintaining - // the operator stack. + OptimizeSparkProcContext procCtx = new OptimizeSparkProcContext(conf, pCtx, inputs, outputs); + + // Run Spark Dynamic Partition Pruning + runDynamicPartitionPruning(procCtx); + + // Annotation OP tree with statistics + runStatsAnnotation(procCtx); + + // Run Join releated optimizations + runJoinOptimizations(procCtx); + + PERF_LOGGER.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_OPTIMIZE_OPERATOR_TREE); + } + + private void runStatsAnnotation(OptimizeSparkProcContext procCtx) throws SemanticException { + new AnnotateWithStatistics().transform(procCtx.getParseContext()); + new AnnotateWithOpTraits().transform(procCtx.getParseContext()); + } + + private void runDynamicPartitionPruning(OptimizeSparkProcContext procCtx) + throws SemanticException { + if (!conf.getBoolVar(HiveConf.ConfVars.SPARK_DYNAMIC_PARTITION_PRUNING)) { + return; + } + + ParseContext parseContext = procCtx.getParseContext(); + Map opRules = new LinkedHashMap(); + opRules.put( + new RuleRegExp(new String("Dynamic Partition Pruning"), + FilterOperator.getOperatorName() + "%"), + new DynamicPartitionPruningOptimization()); + + // The dispatcher fires the processor corresponding to the closest matching + // rule and passes the context along + Dispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx); + GraphWalker ogw = new ForwardWalker(disp); + + List topNodes = new ArrayList(); + topNodes.addAll(parseContext.getTopOps().values()); + ogw.startWalking(topNodes, null); + + // need a new run of the constant folding because we might have created lots + // of "and true and true" conditions. + if(procCtx.getConf().getBoolVar(HiveConf.ConfVars.HIVEOPTCONSTANTPROPAGATION)) { + new ConstantPropagate().transform(parseContext); + } + } + + private void runJoinOptimizations(OptimizeSparkProcContext procCtx) throws SemanticException { + ParseContext pCtx = procCtx.getParseContext(); Map opRules = new LinkedHashMap(); opRules.put(new RuleRegExp("Set parallelism - ReduceSink", - ReduceSinkOperator.getOperatorName() + "%"), - new SetSparkReducerParallelism()); + ReduceSinkOperator.getOperatorName() + "%"), + new SetSparkReducerParallelism()); opRules.put(new TypeRule(JoinOperator.class), new SparkJoinOptimizer(pCtx)); opRules.put(new TypeRule(MapJoinOperator.class), new SparkJoinHintOptimizer(pCtx)); + opRules.put(new RuleRegExp("Disabling Dynamic Partition Pruning By Size", + SparkPartitionPruningSinkOperator.getOperatorName() + "%"), + new SparkRemoveDynamicPruningBySize()); + // The dispatcher fires the processor corresponding to the closest matching // rule and passes the context along Dispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx); @@ -127,7 +178,6 @@ protected void optimizeOperatorPlan(ParseContext pCtx, Set inputs, ArrayList topNodes = new ArrayList(); topNodes.addAll(pCtx.getTopOps().values()); ogw.startWalking(topNodes, null); - PERF_LOGGER.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_OPTIMIZE_OPERATOR_TREE); } /** @@ -138,20 +188,90 @@ protected void generateTaskTree(List> rootTasks, Pa List> mvTask, Set inputs, Set outputs) throws SemanticException { PERF_LOGGER.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_GENERATE_TASK_TREE); - GenSparkUtils.getUtils().resetSequenceNumber(); - ParseContext tempParseContext = getParseContext(pCtx, rootTasks); - GenSparkWork genSparkWork = new GenSparkWork(GenSparkUtils.getUtils()); + GenSparkUtils utils = GenSparkUtils.getUtils(); + utils.resetSequenceNumber(); + ParseContext tempParseContext = getParseContext(pCtx, rootTasks); GenSparkProcContext procCtx = new GenSparkProcContext( conf, tempParseContext, mvTask, rootTasks, inputs, outputs, pCtx.getTopOps()); + // -------------------------------- First Pass ---------------------------------- // + // Identify SparkPartitionPruningSinkOperators, and break OP tree if necessary + + Map opRules = new LinkedHashMap(); + opRules.put(new RuleRegExp("Clone OP tree for PartitionPruningSink", + SparkPartitionPruningSinkOperator.getOperatorName() + "%"), + new SplitOpTreeForDPP()); + + Dispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx); + GraphWalker ogw = new GenSparkWorkWalker(disp, procCtx); + + List topNodes = new ArrayList(); + topNodes.addAll(pCtx.getTopOps().values()); + ogw.startWalking(topNodes, null); + + // -------------------------------- Second Pass ---------------------------------- // + // Process operator tree in two steps: first we process the extra op trees generated + // in the first pass. Then we process the main op tree, and the result task will depend + // on the task generated in the first pass. + topNodes.clear(); + topNodes.addAll(procCtx.topOps.values()); + generateTaskTreeHelper(procCtx, topNodes); + + // If this set is not empty, it means we need to generate a separate task for collecting + // the partitions used. + if (!procCtx.clonedPruningTableScanSet.isEmpty()) { + SparkTask pruningTask = SparkUtilities.createSparkTask(conf); + SparkTask mainTask = procCtx.currentTask; + pruningTask.addDependentTask(procCtx.currentTask); + procCtx.rootTasks.remove(procCtx.currentTask); + procCtx.rootTasks.add(pruningTask); + procCtx.currentTask = pruningTask; + + topNodes.clear(); + topNodes.addAll(procCtx.clonedPruningTableScanSet); + generateTaskTreeHelper(procCtx, topNodes); + + procCtx.currentTask = mainTask; + } + + // -------------------------------- Post Pass ---------------------------------- // + + // we need to clone some operator plans and remove union operators still + for (BaseWork w : procCtx.workWithUnionOperators) { + GenSparkUtils.getUtils().removeUnionOperators(conf, procCtx, w); + } + + // we need to fill MapWork with 'local' work and bucket information for SMB Join. + GenSparkUtils.getUtils().annotateMapWork(procCtx); + + // finally make sure the file sink operators are set up right + for (FileSinkOperator fileSink : procCtx.fileSinkSet) { + GenSparkUtils.getUtils().processFileSink(procCtx, fileSink); + } + + // Process partition pruning sinks + for (Operator prunerSink : procCtx.pruningSinkSet) { + utils.processPartitionPruningSink(procCtx, (SparkPartitionPruningSinkOperator) prunerSink); + } + + PERF_LOGGER.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_GENERATE_TASK_TREE); + } + + private void generateTaskTreeHelper(GenSparkProcContext procCtx, List topNodes) + throws SemanticException { // create a walker which walks the tree in a DFS manner while maintaining // the operator stack. The dispatcher generates the plan from the operator tree Map opRules = new LinkedHashMap(); + GenSparkWork genSparkWork = new GenSparkWork(GenSparkUtils.getUtils()); + opRules.put(new RuleRegExp("Split Work - ReduceSink", ReduceSinkOperator.getOperatorName() + "%"), genSparkWork); + opRules.put(new RuleRegExp("Split Work - SparkPartitionPruningSink", + SparkPartitionPruningSinkOperator.getOperatorName() + "%"), genSparkWork); + opRules.put(new TypeRule(MapJoinOperator.class), new SparkReduceSinkMapJoinProc()); opRules.put(new RuleRegExp("Split Work + Move/Merge - FileSink", @@ -186,8 +306,10 @@ public Object process(Node n, Stack s, * SMBJoinOP * * Some of the other processors are expecting only one traversal beyond SMBJoinOp. - * We need to traverse from the big-table path only, and stop traversing on the small-table path once we reach SMBJoinOp. - * Also add some SMB join information to the context, so we can properly annotate the MapWork later on. + * We need to traverse from the big-table path only, and stop traversing on the + * small-table path once we reach SMBJoinOp. + * Also add some SMB join information to the context, so we can properly annotate + * the MapWork later on. */ opRules.put(new TypeRule(SMBMapJoinOperator.class), new NodeProcessor() { @@ -219,25 +341,8 @@ public Object process(Node currNode, Stack stack, // The dispatcher fires the processor corresponding to the closest matching // rule and passes the context along Dispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx); - List topNodes = new ArrayList(); - topNodes.addAll(pCtx.getTopOps().values()); GraphWalker ogw = new GenSparkWorkWalker(disp, procCtx); ogw.startWalking(topNodes, null); - - // we need to clone some operator plans and remove union operators still - for (BaseWork w: procCtx.workWithUnionOperators) { - GenSparkUtils.getUtils().removeUnionOperators(conf, procCtx, w); - } - - // we need to fill MapWork with 'local' work and bucket information for SMB Join. - GenSparkUtils.getUtils().annotateMapWork(procCtx); - - // finally make sure the file sink operators are set up right - for (FileSinkOperator fileSink: procCtx.fileSinkSet) { - GenSparkUtils.getUtils().processFileSink(procCtx, fileSink); - } - - PERF_LOGGER.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_GENERATE_TASK_TREE); } @Override diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkPartitionPruningSinkOperator.java ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkPartitionPruningSinkOperator.java new file mode 100644 index 0000000..20432c7 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkPartitionPruningSinkOperator.java @@ -0,0 +1,142 @@ +/** + * 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.parse.spark; + +import java.io.BufferedOutputStream; +import java.io.IOException; +import java.io.ObjectOutputStream; +import java.util.Collection; +import java.util.concurrent.Future; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.optimizer.spark.SparkPartitionPruningSinkDesc; +import org.apache.hadoop.hive.ql.plan.api.OperatorType; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.io.DataOutputBuffer; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.util.ReflectionUtils; +import org.apache.hadoop.hive.serde2.Serializer; + +/** + * This operator gets partition info from the upstream operators, and write them + * to HDFS. This will later be read at the driver, and used for pruning the partitions + * for the big table side. + */ +public class SparkPartitionPruningSinkOperator extends Operator { + + @SuppressWarnings("deprecation") + protected transient Serializer serializer; + protected transient DataOutputBuffer buffer; + protected static final Log LOG = LogFactory.getLog(SparkPartitionPruningSinkOperator.class); + + @SuppressWarnings("deprecation") + public Collection> initializeOp(Configuration hconf) throws HiveException { + Collection> result = super.initializeOp(hconf); + serializer = (Serializer) ReflectionUtils.newInstance( + conf.getTable().getDeserializerClass(), null); + buffer = new DataOutputBuffer(); + return result; + } + + @Override + public void process(Object row, int tag) throws HiveException { + ObjectInspector rowInspector = inputObjInspectors[0]; + try { + Writable writableRow = serializer.serialize(row, rowInspector); + writableRow.write(buffer); + } catch (Exception e) { + throw new HiveException(e); + } + } + + @Override + public void closeOp(boolean abort) throws HiveException { + if (!abort) { + try { + flushToFile(); + } catch (Exception e) { + throw new HiveException(e); + } + } + } + + private void flushToFile() throws IOException { + // write an intermediate file to the specified path + // the format of the path is: tmpPath/targetWorkId/sourceWorkId/randInt + Path path = conf.getPath(); + FileSystem fs = path.getFileSystem(this.getConfiguration()); + fs.mkdirs(path); + + while (true) { + path = new Path(path, String.valueOf(Utilities.randGen.nextInt())); + if (!fs.exists(path)) { + break; + } + } + + short numOfRepl = fs.getDefaultReplication(path); + + ObjectOutputStream out = null; + FSDataOutputStream fsout = null; + + try { + fsout = fs.create(path, numOfRepl); + out = new ObjectOutputStream(new BufferedOutputStream(fsout, 4096)); + out.writeUTF(conf.getTargetColumnName()); + buffer.writeTo(out); + } catch (Exception e) { + try { + fs.delete(path, false); + } catch (Exception ex) { + LOG.warn("Exception happened while trying to clean partial file."); + } + throw e; + } finally { + if (out != null) { + LOG.info("Flushed to file: " + path); + out.close(); + } else if (fsout != null) { + fsout.close(); + } + } + } + + @Override + public OperatorType getType() { + return OperatorType.SPARKPRUNINGSINK; + } + + @Override + public String getName() { + return getOperatorName(); + } + + public static String getOperatorName() { + return "SPARKPRUNINGSINK"; + } + +} diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SplitOpTreeForDPP.java ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SplitOpTreeForDPP.java new file mode 100644 index 0000000..c140f67 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SplitOpTreeForDPP.java @@ -0,0 +1,151 @@ +/** + * 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.parse.spark; + +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Set; +import java.util.Stack; + +import com.google.common.base.Preconditions; +import org.apache.hadoop.hive.ql.exec.MapJoinOperator; +import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; +import org.apache.hadoop.hive.ql.exec.TableScanOperator; +import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.exec.spark.SparkUtilities; +import org.apache.hadoop.hive.ql.lib.Node; +import org.apache.hadoop.hive.ql.lib.NodeProcessor; +import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; +import org.apache.hadoop.hive.ql.parse.SemanticException; + +/** + * This processor triggers on SparkPartitionPruningSinkOperator. For a operator tree like + * this: + * + * Original Tree: + * TS TS + * | | + * FIL FIL + * | | \ + * RS RS SEL + * \ / | + * JOIN GBY + * | + * SPARKPRUNINGSINK + * + * It removes the branch containing SPARKPRUNINGSINK from the original operator tree, and splits it into + * two separate trees: + * + * Tree #1: Tree #2: + * TS TS TS + * | | | + * FIL FIL FIL + * | | | + * RS RS SEL + * \ / | + * JOIN GBY + * | + * SPARKPRUNINGSINK + * + * For MapJoinOperator, this optimizer will not do anything - it should be executed within + * the same SparkTask. + */ +public class SplitOpTreeForDPP implements NodeProcessor { + @Override + public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, + Object... nodeOutputs) throws SemanticException { + SparkPartitionPruningSinkOperator pruningSinkOp = (SparkPartitionPruningSinkOperator) nd; + GenSparkProcContext context = (GenSparkProcContext) procCtx; + + // Locate the op where the branch starts + // This is guaranteed to succeed since the branch always follow the pattern + // as shown in the first picture above. + Operator filterOp = pruningSinkOp; + Operator selOp = null; + while (filterOp != null) { + if (filterOp.getNumChild() > 1) { + break; + } else { + selOp = filterOp; + filterOp = filterOp.getParentOperators().get(0); + } + } + + // Check if this is a MapJoin. If so, do not split. + for (Operator childOp : filterOp.getChildOperators()) { + if (childOp instanceof ReduceSinkOperator && + childOp.getChildOperators().get(0) instanceof MapJoinOperator) { + context.pruningSinkSet.add(pruningSinkOp); + return null; + } + } + + List> roots = new LinkedList>(); + collectRoots(roots, pruningSinkOp); + + List> savedChildOps = filterOp.getChildOperators(); + filterOp.setChildOperators(Utilities.makeList(selOp)); + + // Now clone the tree above selOp + List> newRoots = Utilities.cloneOperatorTree(context.parseContext.getConf(), roots); + for (int i = 0; i < roots.size(); i++) { + TableScanOperator newTs = (TableScanOperator) newRoots.get(i); + TableScanOperator oldTs = (TableScanOperator) roots.get(i); + newTs.getConf().setTableMetadata(oldTs.getConf().getTableMetadata()); + } + context.clonedPruningTableScanSet.addAll(newRoots); + + // Restore broken links between operators, and remove the branch from the original tree + filterOp.setChildOperators(savedChildOps); + filterOp.removeChild(selOp); + + // Find the cloned PruningSink and add it to pruningSinkSet + Set> sinkSet = new HashSet>(); + for (Operator root : newRoots) { + SparkUtilities.collectOp(sinkSet, root, SparkPartitionPruningSinkOperator.class); + } + Preconditions.checkArgument(sinkSet.size() == 1, + "AssertionError: expected to only contain one SparkPartitionPruningSinkOperator," + + " but found " + sinkSet.size()); + SparkPartitionPruningSinkOperator clonedPruningSinkOp = + (SparkPartitionPruningSinkOperator) sinkSet.iterator().next(); + clonedPruningSinkOp.getConf().setTableScan(pruningSinkOp.getConf().getTableScan()); + context.pruningSinkSet.add(clonedPruningSinkOp); + + return null; + } + + /** + * Recursively collect all roots (e.g., table scans) that can be reached via this op. + * @param result contains all roots can be reached via op + * @param op the op to examine. + */ + private void collectRoots(List> result, Operator op) { + if (op.getNumParent() == 0) { + result.add(op); + } else { + for (Operator parentOp : op.getParentOperators()) { + collectRoots(result, parentOp); + } + } + } + +} diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java index 3217df2..2cb9257 100644 --- ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java +++ ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java @@ -87,6 +87,8 @@ private Path tmpHDFSPath; + private Path tmpPathForPartitionPruning; + private String inputformat; private String indexIntermediateFile; @@ -455,6 +457,14 @@ public void setTmpHDFSPath(Path tmpHDFSPath) { this.tmpHDFSPath = tmpHDFSPath; } + public Path getTmpPathForPartitionPruning() { + return this.tmpPathForPartitionPruning; + } + + public void setTmpPathForPartitionPruning(Path tmpPathForPartitionPruning) { + this.tmpPathForPartitionPruning = tmpPathForPartitionPruning; + } + public void mergingInto(MapWork mapWork) { // currently, this is sole field affecting mergee task mapWork.useBucketizedHiveInputFormat |= useBucketizedHiveInputFormat; diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java index 9e9a2a2..d27f3ff 100644 --- ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java +++ ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java @@ -103,7 +103,7 @@ private transient TableSample tableSample; - private transient final Table tableMetadata; + private transient Table tableMetadata; public TableScanDesc() { this(null, null); @@ -284,6 +284,10 @@ public Table getTableMetadata() { return tableMetadata; } + public void setTableMetadata(Table tableMetadata) { + this.tableMetadata = tableMetadata; + } + public TableSample getTableSample() { return tableSample; } diff --git ql/src/java/org/apache/hadoop/hive/ql/ppd/SyntheticJoinPredicate.java ql/src/java/org/apache/hadoop/hive/ql/ppd/SyntheticJoinPredicate.java index 363e49e..32af813 100644 --- ql/src/java/org/apache/hadoop/hive/ql/ppd/SyntheticJoinPredicate.java +++ ql/src/java/org/apache/hadoop/hive/ql/ppd/SyntheticJoinPredicate.java @@ -67,8 +67,18 @@ @Override public ParseContext transform(ParseContext pctx) throws SemanticException { - if (!pctx.getConf().getVar(ConfVars.HIVE_EXECUTION_ENGINE).equals("tez") - || !pctx.getConf().getBoolVar(ConfVars.TEZ_DYNAMIC_PARTITION_PRUNING)) { + boolean enabled = false; + String queryEngine = pctx.getConf().getVar(ConfVars.HIVE_EXECUTION_ENGINE); + + if (queryEngine.equals("tez") + && pctx.getConf().getBoolVar(ConfVars.TEZ_DYNAMIC_PARTITION_PRUNING)) { + enabled = true; + } else if ((queryEngine.equals("spark") + && pctx.getConf().getBoolVar(ConfVars.SPARK_DYNAMIC_PARTITION_PRUNING))) { + enabled = true; + } + + if (!enabled) { return pctx; } diff --git ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning.q ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning.q new file mode 100644 index 0000000..8b83ef6 --- /dev/null +++ ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning.q @@ -0,0 +1,180 @@ +set hive.support.sql11.reserved.keywords=false; +set hive.optimize.ppd=true; +set hive.ppd.remove.duplicatefilters=true; +set hive.spark.dynamic.partition.pruning=true; +set hive.optimize.metadataonly=false; +set hive.optimize.index.filter=true; + +-- SORT_QUERY_RESULTS + +select distinct ds from srcpart; +select distinct hr from srcpart; + +EXPLAIN create table srcpart_date as select ds as ds, ds as date from srcpart group by ds; +create table srcpart_date as select ds as ds, ds as date from srcpart group by ds; +create table srcpart_hour as select hr as hr, hr as hour from srcpart group by hr; +create table srcpart_date_hour as select ds as ds, ds as date, hr as hr, hr as hour from srcpart group by ds, hr; +create table srcpart_double_hour as select (hr*2) as hr, hr as hour from srcpart group by hr; + +-- single column, single key +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'; +select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'; +set hive.spark.dynamic.partition.pruning=false; +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'; +select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'; +set hive.spark.dynamic.partition.pruning=true; +select count(*) from srcpart where ds = '2008-04-08'; + +-- multiple sources, single key +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11; +select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11; +set hive.spark.dynamic.partition.pruning=false; +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11; +select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11; +set hive.spark.dynamic.partition.pruning=true; +select count(*) from srcpart where hr = 11 and ds = '2008-04-08'; + +-- multiple columns single source +EXPLAIN select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11; +select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11; +set hive.spark.dynamic.partition.pruning=false; +EXPLAIN select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11; +select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11; +set hive.spark.dynamic.partition.pruning=true; +select count(*) from srcpart where ds = '2008-04-08' and hr = 11; + +-- empty set +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST'; +select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST'; +set hive.spark.dynamic.partition.pruning=false; +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST'; +select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST'; +set hive.spark.dynamic.partition.pruning=true; +select count(*) from srcpart where ds = 'I DONT EXIST'; + +-- expressions +EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11; +select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11; +EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11; +select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11; +set hive.spark.dynamic.partition.pruning=false; +EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11; +select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11; +EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11; +select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11; +set hive.spark.dynamic.partition.pruning=true; +select count(*) from srcpart where hr = 11; +EXPLAIN select count(*) from srcpart join srcpart_double_hour on (cast(srcpart.hr*2 as string) = cast(srcpart_double_hour.hr as string)) where srcpart_double_hour.hour = 11; +select count(*) from srcpart join srcpart_double_hour on (cast(srcpart.hr*2 as string) = cast(srcpart_double_hour.hr as string)) where srcpart_double_hour.hour = 11; +set hive.spark.dynamic.partition.pruning=true; +select count(*) from srcpart where cast(hr as string) = 11; + + +-- parent is reduce tasks +EXPLAIN select count(*) from srcpart join (select ds as ds, ds as date from srcpart group by ds) s on (srcpart.ds = s.ds) where s.date = '2008-04-08'; +select count(*) from srcpart join (select ds as ds, ds as date from srcpart group by ds) s on (srcpart.ds = s.ds) where s.date = '2008-04-08'; +select count(*) from srcpart where ds = '2008-04-08'; + +-- non-equi join +EXPLAIN select count(*) from srcpart, srcpart_date_hour where (srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11) and (srcpart.ds = srcpart_date_hour.ds or srcpart.hr = srcpart_date_hour.hr); +select count(*) from srcpart, srcpart_date_hour where (srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11) and (srcpart.ds = srcpart_date_hour.ds or srcpart.hr = srcpart_date_hour.hr); + +-- old style join syntax +EXPLAIN select count(*) from srcpart, srcpart_date_hour where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11 and srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr; +select count(*) from srcpart, srcpart_date_hour where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11 and srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr; + +-- left join +EXPLAIN select count(*) from srcpart left join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'; +EXPLAIN select count(*) from srcpart_date left join srcpart on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'; + +-- full outer +EXPLAIN select count(*) from srcpart full outer join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'; + +-- with static pruning +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11 and srcpart.hr = 11; +select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11 and srcpart.hr = 11; +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.date = '2008-04-08' and srcpart.hr = 13; +select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.date = '2008-04-08' and srcpart.hr = 13; + +-- union + subquery +EXPLAIN select count(*) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart); +select count(*) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart); +EXPLAIN select distinct(ds) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart); +select distinct(ds) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart); +EXPLAIN select ds from (select distinct(ds) as ds from srcpart union all select distinct(ds) as ds from srcpart) s where s.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart); +select ds from (select distinct(ds) as ds from srcpart union all select distinct(ds) as ds from srcpart) s where s.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart); + +set hive.auto.convert.join=true; +set hive.auto.convert.join.noconditionaltask = true; +set hive.auto.convert.join.noconditionaltask.size = 10000000; + +-- single column, single key +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'; +select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'; +select count(*) from srcpart where ds = '2008-04-08'; + +-- multiple sources, single key +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11; +select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11; +select count(*) from srcpart where hr = 11 and ds = '2008-04-08'; + +-- multiple columns single source +EXPLAIN select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11; +select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11; +select count(*) from srcpart where ds = '2008-04-08' and hr = 11; + +-- empty set +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST'; +-- Disabled until TEZ-1486 is fixed +-- select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST'; + +-- expressions +EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11; +select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11; +EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11; +select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11; +select count(*) from srcpart where hr = 11; + +-- parent is reduce tasks +EXPLAIN select count(*) from srcpart join (select ds as ds, ds as date from srcpart group by ds) s on (srcpart.ds = s.ds) where s.date = '2008-04-08'; +select count(*) from srcpart join (select ds as ds, ds as date from srcpart group by ds) s on (srcpart.ds = s.ds) where s.date = '2008-04-08'; +select count(*) from srcpart where ds = '2008-04-08'; + +-- left join +EXPLAIN select count(*) from srcpart left join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'; +EXPLAIN select count(*) from srcpart_date left join srcpart on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'; + +-- full outer +EXPLAIN select count(*) from srcpart full outer join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08'; + +-- with static pruning +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11 and srcpart.hr = 11; +select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11 and srcpart.hr = 11; +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.date = '2008-04-08' and srcpart.hr = 13; +-- Disabled until TEZ-1486 is fixed +-- select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +-- where srcpart_date.date = '2008-04-08' and srcpart.hr = 13; + +-- union + subquery +EXPLAIN select distinct(ds) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart); +select distinct(ds) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart); + + +drop table srcpart_orc; +drop table srcpart_date; +drop table srcpart_hour; +drop table srcpart_date_hour; +drop table srcpart_double_hour; diff --git ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_2.q ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_2.q new file mode 100644 index 0000000..734f187 --- /dev/null +++ ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_2.q @@ -0,0 +1,118 @@ +set hive.optimize.ppd=true; +set hive.ppd.remove.duplicatefilters=true; +set hive.spark.dynamic.partition.pruning=true; +set hive.optimize.metadataonly=false; +set hive.optimize.index.filter=true; +set hive.auto.convert.join=true; +set hive.auto.convert.join.noconditionaltask = true; +set hive.auto.convert.join.noconditionaltask.size = 10000000; + +-- SORT_QUERY_RESULTS + +create table dim_shops (id int, label string) row format delimited fields terminated by ',' stored as textfile; +load data local inpath '../../data/files/dim_shops.txt' into table dim_shops; + +create table agg_01 (amount decimal) partitioned by (dim_shops_id int) row format delimited fields terminated by ',' stored as textfile; +alter table agg_01 add partition (dim_shops_id = 1); +alter table agg_01 add partition (dim_shops_id = 2); +alter table agg_01 add partition (dim_shops_id = 3); + +load data local inpath '../../data/files/agg_01-p1.txt' into table agg_01 partition (dim_shops_id=1); +load data local inpath '../../data/files/agg_01-p2.txt' into table agg_01 partition (dim_shops_id=2); +load data local inpath '../../data/files/agg_01-p3.txt' into table agg_01 partition (dim_shops_id=3); + +analyze table dim_shops compute statistics; +analyze table agg_01 partition (dim_shops_id) compute statistics; + +select * from dim_shops; +select * from agg_01; + +EXPLAIN SELECT d1.label, count(*), sum(agg.amount) +FROM agg_01 agg, +dim_shops d1 +WHERE agg.dim_shops_id = d1.id +and +d1.label in ('foo', 'bar') +GROUP BY d1.label +ORDER BY d1.label; + +SELECT d1.label, count(*), sum(agg.amount) +FROM agg_01 agg, +dim_shops d1 +WHERE agg.dim_shops_id = d1.id +and +d1.label in ('foo', 'bar') +GROUP BY d1.label +ORDER BY d1.label; + +set hive.spark.dynamic.partition.pruning.max.data.size=1; + +EXPLAIN SELECT d1.label, count(*), sum(agg.amount) +FROM agg_01 agg, +dim_shops d1 +WHERE agg.dim_shops_id = d1.id +and +d1.label in ('foo', 'bar') +GROUP BY d1.label +ORDER BY d1.label; + +SELECT d1.label, count(*), sum(agg.amount) +FROM agg_01 agg, +dim_shops d1 +WHERE agg.dim_shops_id = d1.id +and +d1.label in ('foo', 'bar') +GROUP BY d1.label +ORDER BY d1.label; + +EXPLAIN SELECT d1.label +FROM agg_01 agg, +dim_shops d1 +WHERE agg.dim_shops_id = d1.id; + +SELECT d1.label +FROM agg_01 agg, +dim_shops d1 +WHERE agg.dim_shops_id = d1.id; + +EXPLAIN SELECT agg.amount +FROM agg_01 agg, +dim_shops d1 +WHERE agg.dim_shops_id = d1.id +and agg.dim_shops_id = 1; + +SELECT agg.amount +FROM agg_01 agg, +dim_shops d1 +WHERE agg.dim_shops_id = d1.id +and agg.dim_shops_id = 1; + +set hive.spark.dynamic.partition.pruning.max.data.size=1000000; + +EXPLAIN SELECT d1.label, count(*), sum(agg.amount) +FROM agg_01 agg, +dim_shops d1 +WHERE agg.dim_shops_id = d1.id +and +d1.label in ('foo', 'bar') +GROUP BY d1.label +ORDER BY d1.label; + +SELECT d1.label, count(*), sum(agg.amount) +FROM agg_01 agg, +dim_shops d1 +WHERE agg.dim_shops_id = d1.id +and +d1.label in ('foo', 'bar') +GROUP BY d1.label +ORDER BY d1.label; + + +EXPLAIN +SELECT amount FROM agg_01, dim_shops WHERE dim_shops_id = id AND label = 'foo' +UNION ALL +SELECT amount FROM agg_01, dim_shops WHERE dim_shops_id = id AND label = 'bar'; + +SELECT amount FROM agg_01, dim_shops WHERE dim_shops_id = id AND label = 'foo' +UNION ALL +SELECT amount FROM agg_01, dim_shops WHERE dim_shops_id = id AND label = 'bar'; diff --git ql/src/test/queries/clientpositive/spark_vectorized_dynamic_partition_pruning.q ql/src/test/queries/clientpositive/spark_vectorized_dynamic_partition_pruning.q new file mode 100644 index 0000000..293fcfc --- /dev/null +++ ql/src/test/queries/clientpositive/spark_vectorized_dynamic_partition_pruning.q @@ -0,0 +1,192 @@ +set hive.optimize.ppd=true; +set hive.ppd.remove.duplicatefilters=true; +set hive.spark.dynamic.partition.pruning=true; +set hive.optimize.metadataonly=false; +set hive.optimize.index.filter=true; +set hive.vectorized.execution.enabled=true; + + +select distinct ds from srcpart; +select distinct hr from srcpart; + +EXPLAIN create table srcpart_date as select ds as ds, ds as `date` from srcpart group by ds; +create table srcpart_date stored as orc as select ds as ds, ds as `date` from srcpart group by ds; +create table srcpart_hour stored as orc as select hr as hr, hr as hour from srcpart group by hr; +create table srcpart_date_hour stored as orc as select ds as ds, ds as `date`, hr as hr, hr as hour from srcpart group by ds, hr; +create table srcpart_double_hour stored as orc as select (hr*2) as hr, hr as hour from srcpart group by hr; + +-- single column, single key +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08'; +select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08'; +set hive.spark.dynamic.partition.pruning=false; +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08'; +select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08'; +set hive.spark.dynamic.partition.pruning=true; +select count(*) from srcpart where ds = '2008-04-08'; + +-- multiple sources, single key +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.`date` = '2008-04-08' and srcpart_hour.hour = 11; +select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.`date` = '2008-04-08' and srcpart_hour.hour = 11; +set hive.spark.dynamic.partition.pruning=false; +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.`date` = '2008-04-08' and srcpart_hour.hour = 11; +select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.`date` = '2008-04-08' and srcpart_hour.hour = 11; +set hive.spark.dynamic.partition.pruning=true; +select count(*) from srcpart where hr = 11 and ds = '2008-04-08'; + +-- multiple columns single source +EXPLAIN select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11; +select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11; +set hive.spark.dynamic.partition.pruning=false; +EXPLAIN select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11; +select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11; +set hive.spark.dynamic.partition.pruning=true; +select count(*) from srcpart where ds = '2008-04-08' and hr = 11; + +-- empty set +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = 'I DONT EXIST'; +select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = 'I DONT EXIST'; +set hive.spark.dynamic.partition.pruning=false; +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = 'I DONT EXIST'; +select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = 'I DONT EXIST'; +set hive.spark.dynamic.partition.pruning=true; +select count(*) from srcpart where ds = 'I DONT EXIST'; + +-- expressions +EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11; +select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11; +EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11; +select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11; +set hive.spark.dynamic.partition.pruning=false; +EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11; +select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11; +EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11; +select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11; +set hive.spark.dynamic.partition.pruning=true; +select count(*) from srcpart where hr = 11; +EXPLAIN select count(*) from srcpart join srcpart_double_hour on (cast(srcpart.hr*2 as string) = cast(srcpart_double_hour.hr as string)) where srcpart_double_hour.hour = 11; +select count(*) from srcpart join srcpart_double_hour on (cast(srcpart.hr*2 as string) = cast(srcpart_double_hour.hr as string)) where srcpart_double_hour.hour = 11; +set hive.spark.dynamic.partition.pruning=true; +select count(*) from srcpart where cast(hr as string) = 11; + + +-- parent is reduce tasks +EXPLAIN select count(*) from srcpart join (select ds as ds, ds as `date` from srcpart group by ds) s on (srcpart.ds = s.ds) where s.`date` = '2008-04-08'; +select count(*) from srcpart join (select ds as ds, ds as `date` from srcpart group by ds) s on (srcpart.ds = s.ds) where s.`date` = '2008-04-08'; +select count(*) from srcpart where ds = '2008-04-08'; + +-- non-equi join +EXPLAIN select count(*) from srcpart, srcpart_date_hour where (srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11) and (srcpart.ds = srcpart_date_hour.ds or srcpart.hr = srcpart_date_hour.hr); +select count(*) from srcpart, srcpart_date_hour where (srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11) and (srcpart.ds = srcpart_date_hour.ds or srcpart.hr = srcpart_date_hour.hr); + +-- old style join syntax +EXPLAIN select count(*) from srcpart, srcpart_date_hour where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11 and srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr; +select count(*) from srcpart, srcpart_date_hour where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11 and srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr; + +-- left join +EXPLAIN select count(*) from srcpart left join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08'; +EXPLAIN select count(*) from srcpart_date left join srcpart on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08'; + +-- full outer +EXPLAIN select count(*) from srcpart full outer join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08'; + +-- with static pruning +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.`date` = '2008-04-08' and srcpart_hour.hour = 11 and srcpart.hr = 11; +select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.`date` = '2008-04-08' and srcpart_hour.hour = 11 and srcpart.hr = 11; +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.`date` = '2008-04-08' and srcpart.hr = 13; +select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.`date` = '2008-04-08' and srcpart.hr = 13; + +-- union + subquery +EXPLAIN select count(*) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart); +select count(*) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart); +EXPLAIN select distinct(ds) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart); +select distinct(ds) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart); +EXPLAIN select ds from (select distinct(ds) as ds from srcpart union all select distinct(ds) as ds from srcpart) s where s.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart); +select ds from (select distinct(ds) as ds from srcpart union all select distinct(ds) as ds from srcpart) s where s.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart); + +set hive.auto.convert.join=true; +set hive.auto.convert.join.noconditionaltask = true; +set hive.auto.convert.join.noconditionaltask.size = 10000000; + +-- single column, single key +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08'; +select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08'; +select count(*) from srcpart where ds = '2008-04-08'; + +-- multiple sources, single key +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.`date` = '2008-04-08' and srcpart_hour.hour = 11; +select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.`date` = '2008-04-08' and srcpart_hour.hour = 11; +select count(*) from srcpart where hr = 11 and ds = '2008-04-08'; + +-- multiple columns single source +EXPLAIN select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11; +select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11; +select count(*) from srcpart where ds = '2008-04-08' and hr = 11; + +-- empty set +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = 'I DONT EXIST'; +-- Disabled until TEZ-1486 is fixed +-- select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = 'I DONT EXIST'; + +-- expressions +EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11; +select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11; +EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11; +select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11; +select count(*) from srcpart where hr = 11; + +-- parent is reduce tasks +EXPLAIN select count(*) from srcpart join (select ds as ds, ds as `date` from srcpart group by ds) s on (srcpart.ds = s.ds) where s.`date` = '2008-04-08'; +select count(*) from srcpart join (select ds as ds, ds as `date` from srcpart group by ds) s on (srcpart.ds = s.ds) where s.`date` = '2008-04-08'; +select count(*) from srcpart where ds = '2008-04-08'; + +-- left join +EXPLAIN select count(*) from srcpart left join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08'; +EXPLAIN select count(*) from srcpart_date left join srcpart on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08'; + +-- full outer +EXPLAIN select count(*) from srcpart full outer join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08'; + +-- with static pruning +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.`date` = '2008-04-08' and srcpart_hour.hour = 11 and srcpart.hr = 11; +select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.`date` = '2008-04-08' and srcpart_hour.hour = 11 and srcpart.hr = 11; +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.`date` = '2008-04-08' and srcpart.hr = 13; +-- Disabled until TEZ-1486 is fixed +-- select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +-- where srcpart_date.`date` = '2008-04-08' and srcpart.hr = 13; + +-- union + subquery +EXPLAIN select distinct(ds) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart); +select distinct(ds) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart); + + +-- different file format +create table srcpart_orc (key int, value string) partitioned by (ds string, hr int) stored as orc; + + +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.vectorized.execution.enabled=false; +set hive.exec.max.dynamic.partitions=1000; + +insert into table srcpart_orc partition (ds, hr) select key, value, ds, hr from srcpart; +EXPLAIN select count(*) from srcpart_orc join srcpart_date_hour on (srcpart_orc.ds = srcpart_date_hour.ds and srcpart_orc.hr = srcpart_date_hour.hr) where srcpart_date_hour.hour = 11 and (srcpart_date_hour.`date` = '2008-04-08' or srcpart_date_hour.`date` = '2008-04-09'); +select count(*) from srcpart_orc join srcpart_date_hour on (srcpart_orc.ds = srcpart_date_hour.ds and srcpart_orc.hr = srcpart_date_hour.hr) where srcpart_date_hour.hour = 11 and (srcpart_date_hour.`date` = '2008-04-08' or srcpart_date_hour.`date` = '2008-04-09'); +select count(*) from srcpart where (ds = '2008-04-08' or ds = '2008-04-09') and hr = 11; + +drop table srcpart_orc; +drop table srcpart_date; +drop table srcpart_hour; +drop table srcpart_date_hour; +drop table srcpart_double_hour; diff --git ql/src/test/results/clientpositive/spark/bucket2.q.out ql/src/test/results/clientpositive/spark/bucket2.q.out index 89c3b4c..8bb53d5 100644 --- ql/src/test/results/clientpositive/spark/bucket2.q.out +++ ql/src/test/results/clientpositive/spark/bucket2.q.out @@ -203,14 +203,11 @@ STAGE PLANS: Processor Tree: TableScan alias: s - 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: 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: 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 2fc4855..b25ea05 100644 --- ql/src/test/results/clientpositive/spark/bucket3.q.out +++ ql/src/test/results/clientpositive/spark/bucket3.q.out @@ -226,14 +226,11 @@ STAGE PLANS: Processor Tree: TableScan alias: s - 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: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int), value (type: string), '1' (type: string) outputColumnNames: _col0, _col1, _col2 - 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 44e0f9f..2ad59da 100644 --- ql/src/test/results/clientpositive/spark/bucket4.q.out +++ ql/src/test/results/clientpositive/spark/bucket4.q.out @@ -202,14 +202,11 @@ STAGE PLANS: Processor Tree: TableScan alias: s - 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: 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: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE ListSink PREHOOK: query: select * from bucket4_1 tablesample (bucket 1 out of 2) s diff --git ql/src/test/results/clientpositive/spark/column_access_stats.q.out ql/src/test/results/clientpositive/spark/column_access_stats.q.out index 7879ef1..5803093 100644 --- ql/src/test/results/clientpositive/spark/column_access_stats.q.out +++ ql/src/test/results/clientpositive/spark/column_access_stats.q.out @@ -92,11 +92,9 @@ STAGE PLANS: Processor Tree: TableScan alias: t1 - Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string) outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE ListSink PREHOOK: query: SELECT key FROM (SELECT key, val FROM T1) subq1 @@ -124,11 +122,9 @@ STAGE PLANS: Processor Tree: TableScan alias: t1 - Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string) outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE ListSink PREHOOK: query: SELECT k FROM (SELECT key as k, val as v FROM T1) subq1 diff --git ql/src/test/results/clientpositive/spark/limit_partition_metadataonly.q.out ql/src/test/results/clientpositive/spark/limit_partition_metadataonly.q.out index c6f9039..a34a399 100644 --- ql/src/test/results/clientpositive/spark/limit_partition_metadataonly.q.out +++ ql/src/test/results/clientpositive/spark/limit_partition_metadataonly.q.out @@ -16,11 +16,9 @@ STAGE PLANS: Processor Tree: TableScan alias: srcpart - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: '2008-04-08' (type: string) outputColumnNames: _col0 - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE ListSink PREHOOK: query: select ds from srcpart where hr=11 and ds='2008-04-08' diff --git ql/src/test/results/clientpositive/spark/list_bucket_dml_2.q.java1.7.out ql/src/test/results/clientpositive/spark/list_bucket_dml_2.q.java1.7.out index e38ccf8..603e6bb 100644 --- ql/src/test/results/clientpositive/spark/list_bucket_dml_2.q.java1.7.out +++ ql/src/test/results/clientpositive/spark/list_bucket_dml_2.q.java1.7.out @@ -461,16 +461,13 @@ STAGE PLANS: Processor Tree: TableScan alias: list_bucketing_static_part - Statistics: Num rows: 1000 Data size: 9624 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: ((key = '484') and (value = 'val_484')) (type: boolean) - Statistics: Num rows: 250 Data size: 2406 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: '484' (type: string), 'val_484' (type: string), '2008-04-08' (type: string), '11' (type: string) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 250 Data size: 2406 Basic stats: COMPLETE Column stats: NONE ListSink PREHOOK: query: select * from list_bucketing_static_part where ds = '2008-04-08' and hr = '11' and key = '484' and value = 'val_484' diff --git ql/src/test/results/clientpositive/spark/optimize_nullscan.q.out ql/src/test/results/clientpositive/spark/optimize_nullscan.q.out index a324abc..506d265 100644 --- ql/src/test/results/clientpositive/spark/optimize_nullscan.q.out +++ ql/src/test/results/clientpositive/spark/optimize_nullscan.q.out @@ -43,16 +43,13 @@ STAGE PLANS: Processor Tree: TableScan alias: src - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: false (type: boolean) - Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string) outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE ListSink PREHOOK: query: select key from src where false diff --git ql/src/test/results/clientpositive/spark/pcr.q.out ql/src/test/results/clientpositive/spark/pcr.q.out index efadb1d..fb08f10 100644 --- ql/src/test/results/clientpositive/spark/pcr.q.out +++ ql/src/test/results/clientpositive/spark/pcr.q.out @@ -5461,11 +5461,9 @@ STAGE PLANS: Processor Tree: TableScan alias: srcpart - Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string), value (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE ListSink PREHOOK: query: explain select key,value from srcpart where hr = cast(11 as double) @@ -5482,11 +5480,9 @@ STAGE PLANS: Processor Tree: TableScan alias: srcpart - Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string), value (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE ListSink PREHOOK: query: explain select key,value from srcpart where cast(hr as double) = 11 @@ -5503,10 +5499,8 @@ STAGE PLANS: Processor Tree: TableScan alias: srcpart - Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string), value (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE ListSink diff --git ql/src/test/results/clientpositive/spark/sample3.q.out ql/src/test/results/clientpositive/spark/sample3.q.out index 2fe6b0d..35a4352 100644 --- ql/src/test/results/clientpositive/spark/sample3.q.out +++ ql/src/test/results/clientpositive/spark/sample3.q.out @@ -22,14 +22,11 @@ STAGE PLANS: Processor Tree: TableScan alias: s - Statistics: Num rows: 1000 Data size: 10603 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (((hash(key) & 2147483647) % 5) = 0) (type: boolean) - Statistics: Num rows: 500 Data size: 5301 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int) outputColumnNames: _col0 - Statistics: Num rows: 500 Data size: 5301 Basic stats: COMPLETE Column stats: NONE ListSink PREHOOK: query: SELECT s.key diff --git ql/src/test/results/clientpositive/spark/sample9.q.out ql/src/test/results/clientpositive/spark/sample9.q.out index c9823f7..1a84bd6 100644 --- ql/src/test/results/clientpositive/spark/sample9.q.out +++ ql/src/test/results/clientpositive/spark/sample9.q.out @@ -53,17 +53,14 @@ STAGE PLANS: Processor Tree: TableScan alias: a - Statistics: Num rows: 1000 Data size: 10603 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: true predicate: (((hash(key) & 2147483647) % 2) = 0) (type: boolean) sampleDesc: BUCKET 1 OUT OF 2 - Statistics: Num rows: 500 Data size: 5301 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int), value (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 500 Data size: 5301 Basic stats: COMPLETE Column stats: NONE ListSink PREHOOK: query: SELECT s.* diff --git ql/src/test/results/clientpositive/spark/smb_mapjoin_11.q.out ql/src/test/results/clientpositive/spark/smb_mapjoin_11.q.out index c94cc5b..4d912ca 100644 --- ql/src/test/results/clientpositive/spark/smb_mapjoin_11.q.out +++ ql/src/test/results/clientpositive/spark/smb_mapjoin_11.q.out @@ -1912,17 +1912,14 @@ STAGE PLANS: Processor Tree: TableScan alias: test_table1 - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: true predicate: (((hash(key) & 2147483647) % 16) = 1) (type: boolean) sampleDesc: BUCKET 2 OUT OF 16 - 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: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE ListSink PREHOOK: query: EXPLAIN EXTENDED SELECT * FROM test_table3 TABLESAMPLE(BUCKET 2 OUT OF 16) @@ -2004,17 +2001,14 @@ STAGE PLANS: Processor Tree: TableScan alias: test_table3 - Statistics: Num rows: 1028 Data size: 10968 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: true predicate: (((hash(key) & 2147483647) % 16) = 1) (type: boolean) sampleDesc: BUCKET 2 OUT OF 16 - Statistics: Num rows: 514 Data size: 5484 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: 514 Data size: 5484 Basic stats: COMPLETE Column stats: NONE ListSink PREHOOK: query: SELECT * FROM test_table1 TABLESAMPLE(BUCKET 2 OUT OF 16) diff --git ql/src/test/results/clientpositive/spark/spark_dynamic_partition_pruning.q.out ql/src/test/results/clientpositive/spark/spark_dynamic_partition_pruning.q.out new file mode 100644 index 0000000..a628aae --- /dev/null +++ ql/src/test/results/clientpositive/spark/spark_dynamic_partition_pruning.q.out @@ -0,0 +1,5573 @@ +PREHOOK: query: -- SORT_QUERY_RESULTS + +select distinct ds from srcpart +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +#### A masked pattern was here #### +POSTHOOK: query: -- SORT_QUERY_RESULTS + +select distinct ds from srcpart +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 #### +2008-04-08 +2008-04-09 +PREHOOK: query: select distinct hr from srcpart +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +#### A masked pattern was here #### +POSTHOOK: query: select distinct hr from srcpart +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 #### +11 +12 +PREHOOK: query: EXPLAIN create table srcpart_date as select ds as ds, ds as date from srcpart group by ds +PREHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: query: EXPLAIN create table srcpart_date as select ds as ds, ds as date from srcpart group by ds +POSTHOOK: type: CREATETABLE_AS_SELECT +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + Stage-3 depends on stages: Stage-0 + Stage-2 depends on stages: Stage-3 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 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: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + 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 + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcpart_date + + Stage: Stage-0 + Move Operator + files: + hdfs directory: true +#### A masked pattern was here #### + + Stage: Stage-3 + Create Table Operator: + Create Table + columns: ds string, date string + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat + serde name: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcpart_date + + Stage: Stage-2 + Stats-Aggr Operator + +PREHOOK: query: create table srcpart_date as select ds as ds, ds as date from srcpart group by ds +PREHOOK: type: CREATETABLE_AS_SELECT +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Output: database:default +PREHOOK: Output: default@srcpart_date +POSTHOOK: query: create table srcpart_date as select ds as ds, ds as date from srcpart group by ds +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Output: database:default +POSTHOOK: Output: default@srcpart_date +PREHOOK: query: create table srcpart_hour as select hr as hr, hr as hour from srcpart group by hr +PREHOOK: type: CREATETABLE_AS_SELECT +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Output: database:default +PREHOOK: Output: default@srcpart_hour +POSTHOOK: query: create table srcpart_hour as select hr as hr, hr as hour from srcpart group by hr +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Output: database:default +POSTHOOK: Output: default@srcpart_hour +PREHOOK: query: create table srcpart_date_hour as select ds as ds, ds as date, hr as hr, hr as hour from srcpart group by ds, hr +PREHOOK: type: CREATETABLE_AS_SELECT +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Output: database:default +PREHOOK: Output: default@srcpart_date_hour +POSTHOOK: query: create table srcpart_date_hour as select ds as ds, ds as date, hr as hr, hr as hour from srcpart group by ds, hr +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Output: database:default +POSTHOOK: Output: default@srcpart_date_hour +PREHOOK: query: create table srcpart_double_hour as select (hr*2) as hr, hr as hour from srcpart group by hr +PREHOOK: type: CREATETABLE_AS_SELECT +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Output: database:default +PREHOOK: Output: default@srcpart_double_hour +POSTHOOK: query: create table srcpart_double_hour as select (hr*2) as hr, hr as hour from srcpart group by hr +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Output: database:default +POSTHOOK: Output: default@srcpart_double_hour +PREHOOK: query: -- single column, single key +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08' +PREHOOK: type: QUERY +POSTHOOK: query: -- single column, single key +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08' +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 5 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: ds is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE 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 3 + 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 + 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 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08' +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_date +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_date +#### A masked pattern was here #### +1000 +PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08' +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08' +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 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: ds is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE 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 3 + 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 + 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 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08' +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_date +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_date +#### A masked pattern was here #### +1000 +PREHOOK: query: select count(*) from srcpart where ds = '2008-04-08' +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where ds = '2008-04-08' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +#### A masked pattern was here #### +1000 +PREHOOK: query: -- multiple sources, single key +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11 +PREHOOK: type: QUERY +POSTHOOK: query: -- multiple sources, single key +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 7 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + Map 8 + Map Operator Tree: + TableScan + alias: srcpart_hour + filterExpr: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: hr (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: hr + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + target column name: hr + target work: Map 1 + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + value expressions: hr (type: string) + Map 5 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Map 6 + Map Operator Tree: + TableScan + alias: srcpart_hour + filterExpr: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: hr (type: string) + sort order: + + Map-reduce partition columns: hr (type: string) + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + outputColumnNames: _col3 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col3 (type: string) + sort order: + + Map-reduce partition columns: _col3 (type: string) + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col3 (type: string) + 1 hr (type: string) + Statistics: Num rows: 2420 Data size: 25709 Basic stats: COMPLETE 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 4 + 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 + 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 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11 +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_date +PREHOOK: Input: default@srcpart_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_date +POSTHOOK: Input: default@srcpart_hour +#### A masked pattern was here #### +500 +PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11 +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11 +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 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: (ds is not null and hr is not null) (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + value expressions: hr (type: string) + Map 5 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Map 6 + Map Operator Tree: + TableScan + alias: srcpart_hour + filterExpr: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: hr (type: string) + sort order: + + Map-reduce partition columns: hr (type: string) + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + outputColumnNames: _col3 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col3 (type: string) + sort order: + + Map-reduce partition columns: _col3 (type: string) + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col3 (type: string) + 1 hr (type: string) + Statistics: Num rows: 2420 Data size: 25709 Basic stats: COMPLETE 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 4 + 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 + 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 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11 +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_date +PREHOOK: Input: default@srcpart_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_date +POSTHOOK: Input: default@srcpart_hour +#### A masked pattern was here #### +500 +PREHOOK: query: select count(*) from srcpart where hr = 11 and ds = '2008-04-08' +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where hr = 11 and ds = '2008-04-08' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +500 +PREHOOK: query: -- multiple columns single source +EXPLAIN select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11 +PREHOOK: type: QUERY +POSTHOOK: query: -- multiple columns single source +EXPLAIN select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 5 + Map Operator Tree: + TableScan + alias: srcpart_date_hour + filterExpr: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 4 Data size: 108 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + Map 6 + Map Operator Tree: + TableScan + alias: srcpart_date_hour + filterExpr: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 4 Data size: 108 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: hr (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: hr + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + target column name: hr + target work: Map 1 + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string), hr (type: string) + sort order: ++ + Map-reduce partition columns: ds (type: string), hr (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date_hour + filterExpr: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 4 Data size: 108 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string), hr (type: string) + sort order: ++ + Map-reduce partition columns: ds (type: string), hr (type: string) + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 ds (type: string), hr (type: string) + 1 ds (type: string), hr (type: string) + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE 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 3 + 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 + 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 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11 +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_date_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_date_hour +#### A masked pattern was here #### +500 +PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11 +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11 +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 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: (ds is not null and hr is not null) (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string), hr (type: string) + sort order: ++ + Map-reduce partition columns: ds (type: string), hr (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date_hour + filterExpr: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 4 Data size: 108 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string), hr (type: string) + sort order: ++ + Map-reduce partition columns: ds (type: string), hr (type: string) + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 ds (type: string), hr (type: string) + 1 ds (type: string), hr (type: string) + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE 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 3 + 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 + 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 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11 +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_date_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_date_hour +#### A masked pattern was here #### +500 +PREHOOK: query: select count(*) from srcpart where ds = '2008-04-08' and hr = 11 +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where ds = '2008-04-08' and hr = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +500 +PREHOOK: query: -- empty set +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST' +PREHOOK: type: QUERY +POSTHOOK: query: -- empty set +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST' +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 5 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = 'I DONT EXIST')) (type: boolean) + Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = 'I DONT EXIST')) (type: boolean) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: ds is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = 'I DONT EXIST')) (type: boolean) + Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = 'I DONT EXIST')) (type: boolean) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE 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 3 + 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 + 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 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST' +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_date +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_date +#### A masked pattern was here #### +0 +PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST' +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST' +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 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: ds is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = 'I DONT EXIST')) (type: boolean) + Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = 'I DONT EXIST')) (type: boolean) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE 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 3 + 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 + 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 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST' +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_date +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_date +#### A masked pattern was here #### +0 +PREHOOK: query: select count(*) from srcpart where ds = 'I DONT EXIST' +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where ds = 'I DONT EXIST' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +#### A masked pattern was here #### +0 +PREHOOK: query: -- expressions +EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11 +PREHOOK: type: QUERY +POSTHOOK: query: -- expressions +EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 5 + Map Operator Tree: + TableScan + alias: srcpart_double_hour + filterExpr: (UDFToDouble(UDFToInteger((hr / 2))) is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (UDFToDouble(UDFToInteger((hr / 2))) is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: UDFToDouble(UDFToInteger((hr / 2))) (type: double) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: double) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: UDFToDouble(hr) + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + target column name: hr + target work: Map 1 + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: UDFToDouble(hr) is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: UDFToDouble(hr) is not null (type: boolean) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: UDFToDouble(hr) (type: double) + sort order: + + Map-reduce partition columns: UDFToDouble(hr) (type: double) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_double_hour + filterExpr: (UDFToDouble(UDFToInteger((hr / 2))) is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (UDFToDouble(UDFToInteger((hr / 2))) is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: UDFToDouble(UDFToInteger((hr / 2))) (type: double) + sort order: + + Map-reduce partition columns: UDFToDouble(UDFToInteger((hr / 2))) (type: double) + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 UDFToDouble(hr) (type: double) + 1 UDFToDouble(UDFToInteger((hr / 2))) (type: double) + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE 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 3 + 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 + 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 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11 +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_double_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_double_hour +#### A masked pattern was here #### +1000 +PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11 +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 5 + Map Operator Tree: + TableScan + alias: srcpart_double_hour + filterExpr: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: hr (type: double) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: double) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: (hr * 2) + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + target column name: hr + target work: Map 1 + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: (hr * 2) is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (hr * 2) is not null (type: boolean) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: (hr * 2) (type: double) + sort order: + + Map-reduce partition columns: (hr * 2) (type: double) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_double_hour + filterExpr: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: hr (type: double) + sort order: + + Map-reduce partition columns: hr (type: double) + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 (hr * 2) (type: double) + 1 hr (type: double) + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE 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 3 + 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 + 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 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11 +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_double_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_double_hour +#### A masked pattern was here #### +1000 +PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11 +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11 +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 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: UDFToDouble(hr) is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: UDFToDouble(hr) is not null (type: boolean) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: UDFToDouble(hr) (type: double) + sort order: + + Map-reduce partition columns: UDFToDouble(hr) (type: double) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_double_hour + filterExpr: (UDFToDouble(UDFToInteger((hr / 2))) is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (UDFToDouble(UDFToInteger((hr / 2))) is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: UDFToDouble(UDFToInteger((hr / 2))) (type: double) + sort order: + + Map-reduce partition columns: UDFToDouble(UDFToInteger((hr / 2))) (type: double) + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 UDFToDouble(hr) (type: double) + 1 UDFToDouble(UDFToInteger((hr / 2))) (type: double) + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE 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 3 + 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 + 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 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11 +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_double_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_double_hour +#### A masked pattern was here #### +1000 +PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11 +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11 +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 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: (hr * 2) is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (hr * 2) is not null (type: boolean) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: (hr * 2) (type: double) + sort order: + + Map-reduce partition columns: (hr * 2) (type: double) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_double_hour + filterExpr: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: hr (type: double) + sort order: + + Map-reduce partition columns: hr (type: double) + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 (hr * 2) (type: double) + 1 hr (type: double) + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE 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 3 + 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 + 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 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11 +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_double_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_double_hour +#### A masked pattern was here #### +1000 +PREHOOK: query: select count(*) from srcpart where hr = 11 +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where hr = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 +#### A masked pattern was here #### +1000 +PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_double_hour on (cast(srcpart.hr*2 as string) = cast(srcpart_double_hour.hr as string)) where srcpart_double_hour.hour = 11 +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_double_hour on (cast(srcpart.hr*2 as string) = cast(srcpart_double_hour.hr as string)) where srcpart_double_hour.hour = 11 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 5 + Map Operator Tree: + TableScan + alias: srcpart_double_hour + filterExpr: (UDFToString(hr) is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (UDFToString(hr) is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: UDFToString(hr) (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: UDFToString((hr * 2)) + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + target column name: hr + target work: Map 1 + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: UDFToString((hr * 2)) is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: UDFToString((hr * 2)) is not null (type: boolean) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: UDFToString((hr * 2)) (type: string) + sort order: + + Map-reduce partition columns: UDFToString((hr * 2)) (type: string) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_double_hour + filterExpr: (UDFToString(hr) is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (UDFToString(hr) is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: UDFToString(hr) (type: string) + sort order: + + Map-reduce partition columns: UDFToString(hr) (type: string) + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 UDFToString((hr * 2)) (type: string) + 1 UDFToString(hr) (type: string) + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE 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 3 + 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 + 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 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_double_hour on (cast(srcpart.hr*2 as string) = cast(srcpart_double_hour.hr as string)) where srcpart_double_hour.hour = 11 +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_double_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_double_hour on (cast(srcpart.hr*2 as string) = cast(srcpart_double_hour.hr as string)) where srcpart_double_hour.hour = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_double_hour +#### A masked pattern was here #### +1000 +PREHOOK: query: select count(*) from srcpart where cast(hr as string) = 11 +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where cast(hr as string) = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 +#### A masked pattern was here #### +1000 +Warning: Shuffle Join JOIN[13][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: -- parent is reduce tasks +EXPLAIN select count(*) from srcpart join (select ds as ds, ds as date from srcpart group by ds) s on (srcpart.ds = s.ds) where s.date = '2008-04-08' +PREHOOK: type: QUERY +POSTHOOK: query: -- parent is reduce tasks +EXPLAIN select count(*) from srcpart join (select ds as ds, ds as date from srcpart group by ds) s on (srcpart.ds = s.ds) where s.date = '2008-04-08' +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 5 <- Map 4 (GROUP, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: (ds = '2008-04-08') (type: boolean) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: (ds = '2008-04-08') (type: boolean) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: '2008-04-08' (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1000 Data size: 10624 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: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE 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 3 + 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 + 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 + Reducer 5 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[13][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: select count(*) from srcpart join (select ds as ds, ds as date from srcpart group by ds) s on (srcpart.ds = s.ds) where s.date = '2008-04-08' +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join (select ds as ds, ds as date from srcpart group by ds) s on (srcpart.ds = s.ds) where s.date = '2008-04-08' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +#### A masked pattern was here #### +1000 +PREHOOK: query: select count(*) from srcpart where ds = '2008-04-08' +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where ds = '2008-04-08' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +#### A masked pattern was here #### +1000 +Warning: Shuffle Join JOIN[4][tables = [srcpart, srcpart_date_hour]] in Work 'Reducer 2' is a cross product +PREHOOK: query: -- non-equi join +EXPLAIN select count(*) from srcpart, srcpart_date_hour where (srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11) and (srcpart.ds = srcpart_date_hour.ds or srcpart.hr = srcpart_date_hour.hr) +PREHOOK: type: QUERY +POSTHOOK: query: -- non-equi join +EXPLAIN select count(*) from srcpart, srcpart_date_hour where (srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11) and (srcpart.ds = srcpart_date_hour.ds or srcpart.hr = srcpart_date_hour.hr) +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 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + value expressions: ds (type: string), hr (type: string) + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date_hour + filterExpr: ((date = '2008-04-08') and (hour = 11)) (type: boolean) + Statistics: Num rows: 4 Data size: 108 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((date = '2008-04-08') and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + value expressions: ds (type: string), hr (type: string) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col2, _col3, _col7, _col9 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col2 = _col7) or (_col3 = _col9)) (type: boolean) + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE 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 3 + 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 + 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 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[4][tables = [srcpart, srcpart_date_hour]] in Work 'Reducer 2' is a cross product +PREHOOK: query: select count(*) from srcpart, srcpart_date_hour where (srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11) and (srcpart.ds = srcpart_date_hour.ds or srcpart.hr = srcpart_date_hour.hr) +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_date_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart, srcpart_date_hour where (srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11) and (srcpart.ds = srcpart_date_hour.ds or srcpart.hr = srcpart_date_hour.hr) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_date_hour +#### A masked pattern was here #### +1500 +PREHOOK: query: -- old style join syntax +EXPLAIN select count(*) from srcpart, srcpart_date_hour where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11 and srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr +PREHOOK: type: QUERY +POSTHOOK: query: -- old style join syntax +EXPLAIN select count(*) from srcpart, srcpart_date_hour where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11 and srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 5 + Map Operator Tree: + TableScan + alias: srcpart_date_hour + filterExpr: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 4 Data size: 108 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + Map 6 + Map Operator Tree: + TableScan + alias: srcpart_date_hour + filterExpr: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 4 Data size: 108 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: hr (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: hr + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + target column name: hr + target work: Map 1 + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string), hr (type: string) + sort order: ++ + Map-reduce partition columns: ds (type: string), hr (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date_hour + filterExpr: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 4 Data size: 108 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string), hr (type: string) + sort order: ++ + Map-reduce partition columns: ds (type: string), hr (type: string) + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 ds (type: string), hr (type: string) + 1 ds (type: string), hr (type: string) + outputColumnNames: _col2, _col3, _col7, _col9 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col2 = _col7) and (_col3 = _col9)) (type: boolean) + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE 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 3 + 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 + 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 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart, srcpart_date_hour where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11 and srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_date_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart, srcpart_date_hour where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11 and srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_date_hour +#### A masked pattern was here #### +500 +PREHOOK: query: -- left join +EXPLAIN select count(*) from srcpart left join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08' +PREHOOK: type: QUERY +POSTHOOK: query: -- left join +EXPLAIN select count(*) from srcpart left join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08' +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 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date + Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE + value expressions: date (type: string) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + outputColumnNames: _col8 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col8 = '2008-04-08') (type: boolean) + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE 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 3 + 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 + 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 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select count(*) from srcpart_date left join srcpart on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08' +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart_date left join srcpart on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08' +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 5 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (date = '2008-04-08') (type: boolean) + Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (date = '2008-04-08') (type: boolean) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 4 + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (date = '2008-04-08') (type: boolean) + Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (date = '2008-04-08') (type: boolean) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE 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 3 + 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 + 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 + Processor Tree: + ListSink + +PREHOOK: query: -- full outer +EXPLAIN select count(*) from srcpart full outer join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08' +PREHOOK: type: QUERY +POSTHOOK: query: -- full outer +EXPLAIN select count(*) from srcpart full outer join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08' +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 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date + Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE + value expressions: date (type: string) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Outer Join 0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + outputColumnNames: _col8 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col8 = '2008-04-08') (type: boolean) + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE 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 3 + 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 + 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 + Processor Tree: + ListSink + +PREHOOK: query: -- with static pruning +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11 and srcpart.hr = 11 +PREHOOK: type: QUERY +POSTHOOK: query: -- with static pruning +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11 and srcpart.hr = 11 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 7 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + Map 8 + Map Operator Tree: + TableScan + alias: srcpart_hour + filterExpr: ((hr is not null and (hour = 11)) and (hr = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((hr is not null and (hour = 11)) and (hr = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: '11' (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: hr + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + target column name: hr + target work: Map 1 + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: (hr = 11) (type: boolean) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Map 5 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Map 6 + Map Operator Tree: + TableScan + alias: srcpart_hour + filterExpr: ((hr is not null and (hour = 11)) and (hr = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((hr is not null and (hour = 11)) and (hr = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: '11' (type: string) + sort order: + + Map-reduce partition columns: '11' (type: string) + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: '11' (type: string) + sort order: + + Map-reduce partition columns: '11' (type: string) + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col3 (type: string) + 1 hr (type: string) + Statistics: Num rows: 1210 Data size: 12854 Basic stats: COMPLETE 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 4 + 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 + 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 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11 and srcpart.hr = 11 +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart_date +PREHOOK: Input: default@srcpart_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11 and srcpart.hr = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 +POSTHOOK: Input: default@srcpart_date +POSTHOOK: Input: default@srcpart_hour +#### A masked pattern was here #### +500 +PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.date = '2008-04-08' and srcpart.hr = 13 +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.date = '2008-04-08' and srcpart.hr = 13 +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 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: ((ds is not null and hr is not null) and (hr = 13)) (type: boolean) + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Filter Operator + predicate: ((ds is not null and hr is not null) and (hr = 13)) (type: boolean) + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Map 5 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Map 6 + Map Operator Tree: + TableScan + alias: srcpart_hour + filterExpr: (hr = 13) (type: boolean) + Statistics: Num rows: 2 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (hr = 13) (type: boolean) + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: '13' (type: string) + sort order: + + Map-reduce partition columns: '13' (type: string) + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: '13' (type: string) + sort order: + + Map-reduce partition columns: '13' (type: string) + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col3 (type: string) + 1 hr (type: string) + Statistics: Num rows: 1 Data size: 0 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 4 + 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 + 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 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.date = '2008-04-08' and srcpart.hr = 13 +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart_date +PREHOOK: Input: default@srcpart_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.date = '2008-04-08' and srcpart.hr = 13 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart_date +POSTHOOK: Input: default@srcpart_hour +#### A masked pattern was here #### +0 +PREHOOK: query: -- union + subquery +EXPLAIN select count(*) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +PREHOOK: type: QUERY +POSTHOOK: query: -- union + subquery +EXPLAIN select count(*) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark + Edges: + Reducer 11 <- Map 10 (GROUP, 1) + Reducer 9 <- Map 8 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 10 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(_col0) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Map 8 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(_col0) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Reducer 11 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + Reducer 9 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + + Stage: Stage-1 + Spark + Edges: + Reducer 5 <- Map 4 (GROUP, 1) + Reducer 7 <- Map 6 (GROUP, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2), Reducer 7 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: ds is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 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: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(_col0) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Map 6 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(_col0) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE 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 3 + 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 + 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 + Reducer 5 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 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: 168 Basic stats: COMPLETE Column stats: NONE + Reducer 7 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 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: 168 Basic stats: COMPLETE Column stats: NONE + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +PREHOOK: query: EXPLAIN select distinct(ds) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select distinct(ds) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark + Edges: + Reducer 11 <- Map 10 (GROUP, 1) + Reducer 9 <- Map 8 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 10 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(_col0) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Map 8 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(_col0) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Reducer 11 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + Reducer 9 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + + Stage: Stage-1 + Spark + Edges: + Reducer 5 <- Map 4 (GROUP, 1) + Reducer 7 <- Map 6 (GROUP, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2), Reducer 7 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: ds is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 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: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(_col0) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Map 6 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(_col0) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2200 Data size: 23372 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: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Reducer 3 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1100 Data size: 11686 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 + Reducer 5 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 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: 168 Basic stats: COMPLETE Column stats: NONE + Reducer 7 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 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: 168 Basic stats: COMPLETE Column stats: NONE + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select distinct(ds) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +#### A masked pattern was here #### +POSTHOOK: query: select distinct(ds) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 #### +2008-04-08 +2008-04-09 +PREHOOK: query: EXPLAIN select ds from (select distinct(ds) as ds from srcpart union all select distinct(ds) as ds from srcpart) s where s.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select ds from (select distinct(ds) as ds from srcpart union all select distinct(ds) as ds from srcpart) s where s.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark + Edges: + Reducer 11 <- Map 10 (GROUP, 1) + Reducer 13 <- Map 12 (GROUP, 1) + Reducer 15 <- Map 10 (GROUP, 1) + Reducer 17 <- Map 12 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 10 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(_col0) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Map 12 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(_col0) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Reducer 11 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + Reducer 13 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + Reducer 15 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 4 + Reducer 17 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 4 + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 5 <- Map 1 (GROUP, 2) + Reducer 7 <- Map 6 (GROUP, 1) + Reducer 9 <- Map 8 (GROUP, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2), Reducer 7 (PARTITION-LEVEL SORT, 2), Reducer 9 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: ds is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 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: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map 6 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(_col0) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Map 8 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(_col0) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1000 Data size: 10624 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: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 2200 Data size: 23372 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 + Reducer 5 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1000 Data size: 10624 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: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reducer 7 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 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: 168 Basic stats: COMPLETE Column stats: NONE + Reducer 9 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 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: 168 Basic stats: COMPLETE Column stats: NONE + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select ds from (select distinct(ds) as ds from srcpart union all select distinct(ds) as ds from srcpart) s where s.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +#### A masked pattern was here #### +POSTHOOK: query: select ds from (select distinct(ds) as ds from srcpart union all select distinct(ds) as ds from srcpart) s where s.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 #### +2008-04-08 +2008-04-08 +2008-04-09 +2008-04-09 +PREHOOK: query: -- single column, single key +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08' +PREHOOK: type: QUERY +POSTHOOK: query: -- single column, single key +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08' +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 3 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 ds (type: string) + 1 ds (type: string) + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: ds is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + input vertices: + 1 Map 3 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE 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) + Local Work: + Map Reduce Local Work + 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 + 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 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08' +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_date +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_date +#### A masked pattern was here #### +1000 +PREHOOK: query: select count(*) from srcpart where ds = '2008-04-08' +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where ds = '2008-04-08' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +#### A masked pattern was here #### +1000 +PREHOOK: query: -- multiple sources, single key +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11 +PREHOOK: type: QUERY +POSTHOOK: query: -- multiple sources, single key +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 3 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 ds (type: string) + 1 ds (type: string) + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + Local Work: + Map Reduce Local Work + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_hour + filterExpr: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 _col3 (type: string) + 1 hr (type: string) + Select Operator + expressions: hr (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: hr + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + target column name: hr + target work: Map 1 + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + outputColumnNames: _col3 + input vertices: + 1 Map 3 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col3 (type: string) + 1 hr (type: string) + input vertices: + 1 Map 4 + Statistics: Num rows: 2420 Data size: 25709 Basic stats: COMPLETE 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) + Local Work: + Map Reduce Local Work + 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 + 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 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11 +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_date +PREHOOK: Input: default@srcpart_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_date +POSTHOOK: Input: default@srcpart_hour +#### A masked pattern was here #### +500 +PREHOOK: query: select count(*) from srcpart where hr = 11 and ds = '2008-04-08' +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where hr = 11 and ds = '2008-04-08' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +500 +PREHOOK: query: -- multiple columns single source +EXPLAIN select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11 +PREHOOK: type: QUERY +POSTHOOK: query: -- multiple columns single source +EXPLAIN select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 3 + Map Operator Tree: + TableScan + alias: srcpart_date_hour + filterExpr: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 4 Data size: 108 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 ds (type: string), hr (type: string) + 1 ds (type: string), hr (type: string) + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + Select Operator + expressions: hr (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: hr + Statistics: Num rows: 1 Data size: 27 Basic stats: COMPLETE Column stats: NONE + target column name: hr + target work: Map 1 + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 ds (type: string), hr (type: string) + 1 ds (type: string), hr (type: string) + input vertices: + 1 Map 3 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE 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) + Local Work: + Map Reduce Local Work + 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 + 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 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11 +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_date_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.date = '2008-04-08' and srcpart_date_hour.hour = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_date_hour +#### A masked pattern was here #### +500 +PREHOOK: query: select count(*) from srcpart where ds = '2008-04-08' and hr = 11 +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where ds = '2008-04-08' and hr = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +500 +PREHOOK: query: -- empty set +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST' +PREHOOK: type: QUERY +POSTHOOK: query: -- empty set +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST' +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 3 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = 'I DONT EXIST')) (type: boolean) + Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = 'I DONT EXIST')) (type: boolean) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 ds (type: string) + 1 ds (type: string) + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: ds is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + input vertices: + 1 Map 3 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE 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) + Local Work: + Map Reduce Local Work + 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 + 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 + Processor Tree: + ListSink + +PREHOOK: query: -- Disabled until TEZ-1486 is fixed +-- select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST'; + +-- expressions +EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11 +PREHOOK: type: QUERY +POSTHOOK: query: -- Disabled until TEZ-1486 is fixed +-- select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = 'I DONT EXIST'; + +-- expressions +EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 3 + Map Operator Tree: + TableScan + alias: srcpart_double_hour + filterExpr: (UDFToDouble(UDFToInteger((hr / 2))) is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (UDFToDouble(UDFToInteger((hr / 2))) is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 UDFToDouble(hr) (type: double) + 1 UDFToDouble(UDFToInteger((hr / 2))) (type: double) + Select Operator + expressions: UDFToDouble(UDFToInteger((hr / 2))) (type: double) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: double) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: UDFToDouble(hr) + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + target column name: hr + target work: Map 1 + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: UDFToDouble(hr) is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: UDFToDouble(hr) is not null (type: boolean) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 UDFToDouble(hr) (type: double) + 1 UDFToDouble(UDFToInteger((hr / 2))) (type: double) + input vertices: + 1 Map 3 + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE 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) + Local Work: + Map Reduce Local Work + 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 + 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 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11 +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_double_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_double_hour +#### A masked pattern was here #### +1000 +PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11 +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 3 + Map Operator Tree: + TableScan + alias: srcpart_double_hour + filterExpr: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 (hr * 2) (type: double) + 1 hr (type: double) + Select Operator + expressions: hr (type: double) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: double) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: (hr * 2) + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + target column name: hr + target work: Map 1 + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: (hr * 2) is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (hr * 2) is not null (type: boolean) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 (hr * 2) (type: double) + 1 hr (type: double) + input vertices: + 1 Map 3 + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE 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) + Local Work: + Map Reduce Local Work + 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 + 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 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11 +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_double_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_double_hour +#### A masked pattern was here #### +1000 +PREHOOK: query: select count(*) from srcpart where hr = 11 +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where hr = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 +#### A masked pattern was here #### +1000 +Warning: Map Join MAPJOIN[23][bigTable=?] in task 'Stage-1:MAPRED' is a cross product +PREHOOK: query: -- parent is reduce tasks +EXPLAIN select count(*) from srcpart join (select ds as ds, ds as date from srcpart group by ds) s on (srcpart.ds = s.ds) where s.date = '2008-04-08' +PREHOOK: type: QUERY +POSTHOOK: query: -- parent is reduce tasks +EXPLAIN select count(*) from srcpart join (select ds as ds, ds as date from srcpart group by ds) s on (srcpart.ds = s.ds) where s.date = '2008-04-08' +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark + Edges: + Reducer 4 <- Map 3 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 3 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: (ds = '2008-04-08') (type: boolean) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: '2008-04-08' (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1000 Data size: 10624 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: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Reducer 4 + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 + 1 + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: (ds = '2008-04-08') (type: boolean) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + input vertices: + 1 Reducer 4 + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE 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) + Local Work: + Map Reduce Local Work + 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 + 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 + Processor Tree: + ListSink + +Warning: Map Join MAPJOIN[23][bigTable=?] in task 'Stage-1:MAPRED' is a cross product +PREHOOK: query: select count(*) from srcpart join (select ds as ds, ds as date from srcpart group by ds) s on (srcpart.ds = s.ds) where s.date = '2008-04-08' +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join (select ds as ds, ds as date from srcpart group by ds) s on (srcpart.ds = s.ds) where s.date = '2008-04-08' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +#### A masked pattern was here #### +1000 +PREHOOK: query: select count(*) from srcpart where ds = '2008-04-08' +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where ds = '2008-04-08' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +#### A masked pattern was here #### +1000 +PREHOOK: query: -- left join +EXPLAIN select count(*) from srcpart left join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08' +PREHOOK: type: QUERY +POSTHOOK: query: -- left join +EXPLAIN select count(*) from srcpart left join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08' +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 3 + Map Operator Tree: + TableScan + alias: srcpart_date + Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 ds (type: string) + 1 ds (type: string) + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Left Outer Join0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + outputColumnNames: _col8 + input vertices: + 1 Map 3 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col8 = '2008-04-08') (type: boolean) + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE 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) + Local Work: + Map Reduce Local Work + 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 + 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 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select count(*) from srcpart_date left join srcpart on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08' +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart_date left join srcpart on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08' +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-3 depends on stages: Stage-2 + Stage-1 depends on stages: Stage-3 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (date = '2008-04-08') (type: boolean) + Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (date = '2008-04-08') (type: boolean) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 3 + + Stage: Stage-3 + Spark +#### A masked pattern was here #### + Vertices: + Map 3 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 ds (type: string) + 1 ds (type: string) + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (date = '2008-04-08') (type: boolean) + Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (date = '2008-04-08') (type: boolean) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Left Outer Join0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + input vertices: + 1 Map 3 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE 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) + Local Work: + Map Reduce Local Work + 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 + 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 + Processor Tree: + ListSink + +PREHOOK: query: -- full outer +EXPLAIN select count(*) from srcpart full outer join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08' +PREHOOK: type: QUERY +POSTHOOK: query: -- full outer +EXPLAIN select count(*) from srcpart full outer join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.date = '2008-04-08' +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 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date + Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE + value expressions: date (type: string) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Outer Join 0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + outputColumnNames: _col8 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col8 = '2008-04-08') (type: boolean) + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE 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 3 + 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 + 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 + Processor Tree: + ListSink + +PREHOOK: query: -- with static pruning +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11 and srcpart.hr = 11 +PREHOOK: type: QUERY +POSTHOOK: query: -- with static pruning +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11 and srcpart.hr = 11 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 3 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 ds (type: string) + 1 ds (type: string) + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + Local Work: + Map Reduce Local Work + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_hour + filterExpr: ((hr is not null and (hour = 11)) and (hr = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((hr is not null and (hour = 11)) and (hr = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 '11' (type: string) + 1 '11' (type: string) + Select Operator + expressions: '11' (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: hr + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + target column name: hr + target work: Map 1 + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: (hr = 11) (type: boolean) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + input vertices: + 1 Map 3 + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 '11' (type: string) + 1 '11' (type: string) + input vertices: + 1 Map 4 + Statistics: Num rows: 1210 Data size: 12854 Basic stats: COMPLETE 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) + Local Work: + Map Reduce Local Work + 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 + 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 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11 and srcpart.hr = 11 +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart_date +PREHOOK: Input: default@srcpart_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.date = '2008-04-08' and srcpart_hour.hour = 11 and srcpart.hr = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 +POSTHOOK: Input: default@srcpart_date +POSTHOOK: Input: default@srcpart_hour +#### A masked pattern was here #### +500 +PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.date = '2008-04-08' and srcpart.hr = 13 +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.date = '2008-04-08' and srcpart.hr = 13 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-3 is a root stage + Stage-2 depends on stages: Stage-3 + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-3 + Spark +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: ((ds is not null and hr is not null) and (hr = 13)) (type: boolean) + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Filter Operator + predicate: ((ds is not null and hr is not null) and (hr = 13)) (type: boolean) + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 ds (type: string) + 1 ds (type: string) + Local Work: + Map Reduce Local Work + + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 2 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 2 Data size: 42 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 1 Data size: 21 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + input vertices: + 0 Map 1 + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 '13' (type: string) + 1 '13' (type: string) + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark + Edges: + Reducer 4 <- Map 3 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 3 + Map Operator Tree: + TableScan + alias: srcpart_hour + filterExpr: (hr = 13) (type: boolean) + Statistics: Num rows: 2 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (hr = 13) (type: boolean) + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 '13' (type: string) + 1 '13' (type: string) + input vertices: + 0 Map 2 + Statistics: Num rows: 1 Data size: 0 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) + Local Work: + Map Reduce Local Work + Reducer 4 + 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 + 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 + Processor Tree: + ListSink + +PREHOOK: query: -- Disabled until TEZ-1486 is fixed +-- select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +-- where srcpart_date.date = '2008-04-08' and srcpart.hr = 13; + +-- union + subquery +EXPLAIN select distinct(ds) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +PREHOOK: type: QUERY +POSTHOOK: query: -- Disabled until TEZ-1486 is fixed +-- select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +-- where srcpart_date.date = '2008-04-08' and srcpart.hr = 13; + +-- union + subquery +EXPLAIN select distinct(ds) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark + Edges: + Reducer 11 <- Map 10 (GROUP, 1) + Reducer 9 <- Map 8 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 10 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(_col0) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Map 8 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(_col0) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Reducer 11 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + Reducer 9 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + + Stage: Stage-1 + Spark + Edges: + Reducer 5 <- Map 4 (GROUP, 1) + Reducer 7 <- Map 6 (GROUP, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2), Reducer 7 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: ds is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 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: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(_col0) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Map 6 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(_col0) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2200 Data size: 23372 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: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Reducer 3 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1100 Data size: 11686 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 + Reducer 5 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 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: 168 Basic stats: COMPLETE Column stats: NONE + Reducer 7 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 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: 168 Basic stats: COMPLETE Column stats: NONE + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select distinct(ds) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +#### A masked pattern was here #### +POSTHOOK: query: select distinct(ds) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 #### +2008-04-08 +2008-04-09 +PREHOOK: query: drop table srcpart_orc +PREHOOK: type: DROPTABLE +POSTHOOK: query: drop table srcpart_orc +POSTHOOK: type: DROPTABLE +PREHOOK: query: drop table srcpart_date +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@srcpart_date +PREHOOK: Output: default@srcpart_date +POSTHOOK: query: drop table srcpart_date +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@srcpart_date +POSTHOOK: Output: default@srcpart_date +PREHOOK: query: drop table srcpart_hour +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@srcpart_hour +PREHOOK: Output: default@srcpart_hour +POSTHOOK: query: drop table srcpart_hour +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@srcpart_hour +POSTHOOK: Output: default@srcpart_hour +PREHOOK: query: drop table srcpart_date_hour +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@srcpart_date_hour +PREHOOK: Output: default@srcpart_date_hour +POSTHOOK: query: drop table srcpart_date_hour +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@srcpart_date_hour +POSTHOOK: Output: default@srcpart_date_hour +PREHOOK: query: drop table srcpart_double_hour +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@srcpart_double_hour +PREHOOK: Output: default@srcpart_double_hour +POSTHOOK: query: drop table srcpart_double_hour +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@srcpart_double_hour +POSTHOOK: Output: default@srcpart_double_hour diff --git ql/src/test/results/clientpositive/spark/spark_dynamic_partition_pruning_2.q.out ql/src/test/results/clientpositive/spark/spark_dynamic_partition_pruning_2.q.out new file mode 100644 index 0000000..4e62a3b --- /dev/null +++ ql/src/test/results/clientpositive/spark/spark_dynamic_partition_pruning_2.q.out @@ -0,0 +1,1015 @@ +PREHOOK: query: -- SORT_QUERY_RESULTS + +create table dim_shops (id int, label string) row format delimited fields terminated by ',' stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@dim_shops +POSTHOOK: query: -- SORT_QUERY_RESULTS + +create table dim_shops (id int, label string) row format delimited fields terminated by ',' stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@dim_shops +PREHOOK: query: load data local inpath '../../data/files/dim_shops.txt' into table dim_shops +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@dim_shops +POSTHOOK: query: load data local inpath '../../data/files/dim_shops.txt' into table dim_shops +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@dim_shops +PREHOOK: query: create table agg_01 (amount decimal) partitioned by (dim_shops_id int) row format delimited fields terminated by ',' stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@agg_01 +POSTHOOK: query: create table agg_01 (amount decimal) partitioned by (dim_shops_id int) row format delimited fields terminated by ',' stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@agg_01 +PREHOOK: query: alter table agg_01 add partition (dim_shops_id = 1) +PREHOOK: type: ALTERTABLE_ADDPARTS +PREHOOK: Output: default@agg_01 +POSTHOOK: query: alter table agg_01 add partition (dim_shops_id = 1) +POSTHOOK: type: ALTERTABLE_ADDPARTS +POSTHOOK: Output: default@agg_01 +POSTHOOK: Output: default@agg_01@dim_shops_id=1 +PREHOOK: query: alter table agg_01 add partition (dim_shops_id = 2) +PREHOOK: type: ALTERTABLE_ADDPARTS +PREHOOK: Output: default@agg_01 +POSTHOOK: query: alter table agg_01 add partition (dim_shops_id = 2) +POSTHOOK: type: ALTERTABLE_ADDPARTS +POSTHOOK: Output: default@agg_01 +POSTHOOK: Output: default@agg_01@dim_shops_id=2 +PREHOOK: query: alter table agg_01 add partition (dim_shops_id = 3) +PREHOOK: type: ALTERTABLE_ADDPARTS +PREHOOK: Output: default@agg_01 +POSTHOOK: query: alter table agg_01 add partition (dim_shops_id = 3) +POSTHOOK: type: ALTERTABLE_ADDPARTS +POSTHOOK: Output: default@agg_01 +POSTHOOK: Output: default@agg_01@dim_shops_id=3 +PREHOOK: query: load data local inpath '../../data/files/agg_01-p1.txt' into table agg_01 partition (dim_shops_id=1) +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@agg_01@dim_shops_id=1 +POSTHOOK: query: load data local inpath '../../data/files/agg_01-p1.txt' into table agg_01 partition (dim_shops_id=1) +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@agg_01@dim_shops_id=1 +PREHOOK: query: load data local inpath '../../data/files/agg_01-p2.txt' into table agg_01 partition (dim_shops_id=2) +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@agg_01@dim_shops_id=2 +POSTHOOK: query: load data local inpath '../../data/files/agg_01-p2.txt' into table agg_01 partition (dim_shops_id=2) +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@agg_01@dim_shops_id=2 +PREHOOK: query: load data local inpath '../../data/files/agg_01-p3.txt' into table agg_01 partition (dim_shops_id=3) +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@agg_01@dim_shops_id=3 +POSTHOOK: query: load data local inpath '../../data/files/agg_01-p3.txt' into table agg_01 partition (dim_shops_id=3) +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@agg_01@dim_shops_id=3 +PREHOOK: query: analyze table dim_shops compute statistics +PREHOOK: type: QUERY +PREHOOK: Input: default@dim_shops +PREHOOK: Output: default@dim_shops +POSTHOOK: query: analyze table dim_shops compute statistics +POSTHOOK: type: QUERY +POSTHOOK: Input: default@dim_shops +POSTHOOK: Output: default@dim_shops +PREHOOK: query: analyze table agg_01 partition (dim_shops_id) compute statistics +PREHOOK: type: QUERY +PREHOOK: Input: default@agg_01 +PREHOOK: Input: default@agg_01@dim_shops_id=1 +PREHOOK: Input: default@agg_01@dim_shops_id=2 +PREHOOK: Input: default@agg_01@dim_shops_id=3 +PREHOOK: Output: default@agg_01 +PREHOOK: Output: default@agg_01@dim_shops_id=1 +PREHOOK: Output: default@agg_01@dim_shops_id=2 +PREHOOK: Output: default@agg_01@dim_shops_id=3 +POSTHOOK: query: analyze table agg_01 partition (dim_shops_id) compute statistics +POSTHOOK: type: QUERY +POSTHOOK: Input: default@agg_01 +POSTHOOK: Input: default@agg_01@dim_shops_id=1 +POSTHOOK: Input: default@agg_01@dim_shops_id=2 +POSTHOOK: Input: default@agg_01@dim_shops_id=3 +POSTHOOK: Output: default@agg_01 +POSTHOOK: Output: default@agg_01@dim_shops_id=1 +POSTHOOK: Output: default@agg_01@dim_shops_id=2 +POSTHOOK: Output: default@agg_01@dim_shops_id=3 +PREHOOK: query: select * from dim_shops +PREHOOK: type: QUERY +PREHOOK: Input: default@dim_shops +#### A masked pattern was here #### +POSTHOOK: query: select * from dim_shops +POSTHOOK: type: QUERY +POSTHOOK: Input: default@dim_shops +#### A masked pattern was here #### +1 foo +2 bar +3 baz +PREHOOK: query: select * from agg_01 +PREHOOK: type: QUERY +PREHOOK: Input: default@agg_01 +PREHOOK: Input: default@agg_01@dim_shops_id=1 +PREHOOK: Input: default@agg_01@dim_shops_id=2 +PREHOOK: Input: default@agg_01@dim_shops_id=3 +#### A masked pattern was here #### +POSTHOOK: query: select * from agg_01 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@agg_01 +POSTHOOK: Input: default@agg_01@dim_shops_id=1 +POSTHOOK: Input: default@agg_01@dim_shops_id=2 +POSTHOOK: Input: default@agg_01@dim_shops_id=3 +#### A masked pattern was here #### +1 1 +2 1 +3 1 +4 2 +5 2 +6 2 +7 3 +8 3 +9 3 +PREHOOK: query: EXPLAIN SELECT d1.label, count(*), sum(agg.amount) +FROM agg_01 agg, +dim_shops d1 +WHERE agg.dim_shops_id = d1.id +and +d1.label in ('foo', 'bar') +GROUP BY d1.label +ORDER BY d1.label +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN SELECT d1.label, count(*), sum(agg.amount) +FROM agg_01 agg, +dim_shops d1 +WHERE agg.dim_shops_id = d1.id +and +d1.label in ('foo', 'bar') +GROUP BY d1.label +ORDER BY d1.label +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 4 + Map Operator Tree: + TableScan + alias: d1 + filterExpr: (id is not null and (label) IN ('foo', 'bar')) (type: boolean) + Statistics: Num rows: 3 Data size: 15 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (id is not null and (label) IN ('foo', 'bar')) (type: boolean) + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 dim_shops_id (type: int) + 1 id (type: int) + Select Operator + expressions: id (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: dim_shops_id + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + target column name: dim_shops_id + target work: Map 1 + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 3 <- Reducer 2 (SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: agg + filterExpr: dim_shops_id is not null (type: boolean) + Statistics: Num rows: 9 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 dim_shops_id (type: int) + 1 id (type: int) + outputColumnNames: _col0, _col1, _col5, _col6 + input vertices: + 1 Map 4 + Statistics: Num rows: 9 Data size: 29 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col1 = _col5) and (_col6) IN ('foo', 'bar')) (type: boolean) + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col6 (type: string), _col0 (type: decimal(10,0)) + outputColumnNames: _col6, _col0 + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), sum(_col0) + keys: _col6 (type: string) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 2 Data size: 6 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: 6 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: decimal(20,0)) + Local Work: + Map Reduce Local Work + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), sum(VALUE._col1) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: decimal(20,0)) + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: bigint), VALUE._col1 (type: decimal(20,0)) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 3 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 + Processor Tree: + ListSink + +PREHOOK: query: SELECT d1.label, count(*), sum(agg.amount) +FROM agg_01 agg, +dim_shops d1 +WHERE agg.dim_shops_id = d1.id +and +d1.label in ('foo', 'bar') +GROUP BY d1.label +ORDER BY d1.label +PREHOOK: type: QUERY +PREHOOK: Input: default@agg_01 +PREHOOK: Input: default@agg_01@dim_shops_id=1 +PREHOOK: Input: default@agg_01@dim_shops_id=2 +PREHOOK: Input: default@agg_01@dim_shops_id=3 +PREHOOK: Input: default@dim_shops +#### A masked pattern was here #### +POSTHOOK: query: SELECT d1.label, count(*), sum(agg.amount) +FROM agg_01 agg, +dim_shops d1 +WHERE agg.dim_shops_id = d1.id +and +d1.label in ('foo', 'bar') +GROUP BY d1.label +ORDER BY d1.label +POSTHOOK: type: QUERY +POSTHOOK: Input: default@agg_01 +POSTHOOK: Input: default@agg_01@dim_shops_id=1 +POSTHOOK: Input: default@agg_01@dim_shops_id=2 +POSTHOOK: Input: default@agg_01@dim_shops_id=3 +POSTHOOK: Input: default@dim_shops +#### A masked pattern was here #### +bar 3 15 +foo 3 6 +PREHOOK: query: EXPLAIN SELECT d1.label, count(*), sum(agg.amount) +FROM agg_01 agg, +dim_shops d1 +WHERE agg.dim_shops_id = d1.id +and +d1.label in ('foo', 'bar') +GROUP BY d1.label +ORDER BY d1.label +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN SELECT d1.label, count(*), sum(agg.amount) +FROM agg_01 agg, +dim_shops d1 +WHERE agg.dim_shops_id = d1.id +and +d1.label in ('foo', 'bar') +GROUP BY d1.label +ORDER BY d1.label +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 4 + Map Operator Tree: + TableScan + alias: d1 + filterExpr: (id is not null and (label) IN ('foo', 'bar')) (type: boolean) + Statistics: Num rows: 3 Data size: 15 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (id is not null and (label) IN ('foo', 'bar')) (type: boolean) + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 dim_shops_id (type: int) + 1 id (type: int) + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 3 <- Reducer 2 (SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: agg + filterExpr: dim_shops_id is not null (type: boolean) + Statistics: Num rows: 9 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 dim_shops_id (type: int) + 1 id (type: int) + outputColumnNames: _col0, _col1, _col5, _col6 + input vertices: + 1 Map 4 + Statistics: Num rows: 9 Data size: 29 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col1 = _col5) and (_col6) IN ('foo', 'bar')) (type: boolean) + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col6 (type: string), _col0 (type: decimal(10,0)) + outputColumnNames: _col6, _col0 + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), sum(_col0) + keys: _col6 (type: string) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 2 Data size: 6 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: 6 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: decimal(20,0)) + Local Work: + Map Reduce Local Work + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), sum(VALUE._col1) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: decimal(20,0)) + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: bigint), VALUE._col1 (type: decimal(20,0)) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 3 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 + Processor Tree: + ListSink + +PREHOOK: query: SELECT d1.label, count(*), sum(agg.amount) +FROM agg_01 agg, +dim_shops d1 +WHERE agg.dim_shops_id = d1.id +and +d1.label in ('foo', 'bar') +GROUP BY d1.label +ORDER BY d1.label +PREHOOK: type: QUERY +PREHOOK: Input: default@agg_01 +PREHOOK: Input: default@agg_01@dim_shops_id=1 +PREHOOK: Input: default@agg_01@dim_shops_id=2 +PREHOOK: Input: default@agg_01@dim_shops_id=3 +PREHOOK: Input: default@dim_shops +#### A masked pattern was here #### +POSTHOOK: query: SELECT d1.label, count(*), sum(agg.amount) +FROM agg_01 agg, +dim_shops d1 +WHERE agg.dim_shops_id = d1.id +and +d1.label in ('foo', 'bar') +GROUP BY d1.label +ORDER BY d1.label +POSTHOOK: type: QUERY +POSTHOOK: Input: default@agg_01 +POSTHOOK: Input: default@agg_01@dim_shops_id=1 +POSTHOOK: Input: default@agg_01@dim_shops_id=2 +POSTHOOK: Input: default@agg_01@dim_shops_id=3 +POSTHOOK: Input: default@dim_shops +#### A masked pattern was here #### +bar 3 15 +foo 3 6 +PREHOOK: query: EXPLAIN SELECT d1.label +FROM agg_01 agg, +dim_shops d1 +WHERE agg.dim_shops_id = d1.id +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN SELECT d1.label +FROM agg_01 agg, +dim_shops d1 +WHERE agg.dim_shops_id = d1.id +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 2 + Map Operator Tree: + TableScan + alias: d1 + filterExpr: id is not null (type: boolean) + Statistics: Num rows: 3 Data size: 15 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: id is not null (type: boolean) + Statistics: Num rows: 2 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 dim_shops_id (type: int) + 1 id (type: int) + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: agg + filterExpr: dim_shops_id is not null (type: boolean) + Statistics: Num rows: 9 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 dim_shops_id (type: int) + 1 id (type: int) + outputColumnNames: _col1, _col5, _col6 + input vertices: + 1 Map 2 + Statistics: Num rows: 9 Data size: 29 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col1 = _col5) (type: boolean) + Statistics: Num rows: 4 Data size: 12 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col6 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 4 Data size: 12 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 4 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 + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Local Work: + Map Reduce Local Work + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: SELECT d1.label +FROM agg_01 agg, +dim_shops d1 +WHERE agg.dim_shops_id = d1.id +PREHOOK: type: QUERY +PREHOOK: Input: default@agg_01 +PREHOOK: Input: default@agg_01@dim_shops_id=1 +PREHOOK: Input: default@agg_01@dim_shops_id=2 +PREHOOK: Input: default@agg_01@dim_shops_id=3 +PREHOOK: Input: default@dim_shops +#### A masked pattern was here #### +POSTHOOK: query: SELECT d1.label +FROM agg_01 agg, +dim_shops d1 +WHERE agg.dim_shops_id = d1.id +POSTHOOK: type: QUERY +POSTHOOK: Input: default@agg_01 +POSTHOOK: Input: default@agg_01@dim_shops_id=1 +POSTHOOK: Input: default@agg_01@dim_shops_id=2 +POSTHOOK: Input: default@agg_01@dim_shops_id=3 +POSTHOOK: Input: default@dim_shops +#### A masked pattern was here #### +bar +bar +bar +baz +baz +baz +foo +foo +foo +PREHOOK: query: EXPLAIN SELECT agg.amount +FROM agg_01 agg, +dim_shops d1 +WHERE agg.dim_shops_id = d1.id +and agg.dim_shops_id = 1 +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN SELECT agg.amount +FROM agg_01 agg, +dim_shops d1 +WHERE agg.dim_shops_id = d1.id +and agg.dim_shops_id = 1 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 2 + Map Operator Tree: + TableScan + alias: d1 + filterExpr: (id = 1) (type: boolean) + Statistics: Num rows: 3 Data size: 15 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (id = 1) (type: boolean) + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 1 (type: int) + 1 1 (type: int) + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: agg + filterExpr: (dim_shops_id = 1) (type: boolean) + Statistics: Num rows: 3 Data size: 9 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 1 (type: int) + 1 1 (type: int) + outputColumnNames: _col0 + input vertices: + 1 Map 2 + Statistics: Num rows: 3 Data size: 9 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 3 Data size: 9 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 + Local Work: + Map Reduce Local Work + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: SELECT agg.amount +FROM agg_01 agg, +dim_shops d1 +WHERE agg.dim_shops_id = d1.id +and agg.dim_shops_id = 1 +PREHOOK: type: QUERY +PREHOOK: Input: default@agg_01 +PREHOOK: Input: default@agg_01@dim_shops_id=1 +PREHOOK: Input: default@dim_shops +#### A masked pattern was here #### +POSTHOOK: query: SELECT agg.amount +FROM agg_01 agg, +dim_shops d1 +WHERE agg.dim_shops_id = d1.id +and agg.dim_shops_id = 1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@agg_01 +POSTHOOK: Input: default@agg_01@dim_shops_id=1 +POSTHOOK: Input: default@dim_shops +#### A masked pattern was here #### +1 +2 +3 +PREHOOK: query: EXPLAIN SELECT d1.label, count(*), sum(agg.amount) +FROM agg_01 agg, +dim_shops d1 +WHERE agg.dim_shops_id = d1.id +and +d1.label in ('foo', 'bar') +GROUP BY d1.label +ORDER BY d1.label +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN SELECT d1.label, count(*), sum(agg.amount) +FROM agg_01 agg, +dim_shops d1 +WHERE agg.dim_shops_id = d1.id +and +d1.label in ('foo', 'bar') +GROUP BY d1.label +ORDER BY d1.label +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 4 + Map Operator Tree: + TableScan + alias: d1 + filterExpr: (id is not null and (label) IN ('foo', 'bar')) (type: boolean) + Statistics: Num rows: 3 Data size: 15 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (id is not null and (label) IN ('foo', 'bar')) (type: boolean) + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 dim_shops_id (type: int) + 1 id (type: int) + Select Operator + expressions: id (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: dim_shops_id + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + target column name: dim_shops_id + target work: Map 1 + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 3 <- Reducer 2 (SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: agg + filterExpr: dim_shops_id is not null (type: boolean) + Statistics: Num rows: 9 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 dim_shops_id (type: int) + 1 id (type: int) + outputColumnNames: _col0, _col1, _col5, _col6 + input vertices: + 1 Map 4 + Statistics: Num rows: 9 Data size: 29 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col1 = _col5) and (_col6) IN ('foo', 'bar')) (type: boolean) + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col6 (type: string), _col0 (type: decimal(10,0)) + outputColumnNames: _col6, _col0 + Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(), sum(_col0) + keys: _col6 (type: string) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 2 Data size: 6 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: 6 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: decimal(20,0)) + Local Work: + Map Reduce Local Work + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0), sum(VALUE._col1) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint), _col2 (type: decimal(20,0)) + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: bigint), VALUE._col1 (type: decimal(20,0)) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 3 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 + Processor Tree: + ListSink + +PREHOOK: query: SELECT d1.label, count(*), sum(agg.amount) +FROM agg_01 agg, +dim_shops d1 +WHERE agg.dim_shops_id = d1.id +and +d1.label in ('foo', 'bar') +GROUP BY d1.label +ORDER BY d1.label +PREHOOK: type: QUERY +PREHOOK: Input: default@agg_01 +PREHOOK: Input: default@agg_01@dim_shops_id=1 +PREHOOK: Input: default@agg_01@dim_shops_id=2 +PREHOOK: Input: default@agg_01@dim_shops_id=3 +PREHOOK: Input: default@dim_shops +#### A masked pattern was here #### +POSTHOOK: query: SELECT d1.label, count(*), sum(agg.amount) +FROM agg_01 agg, +dim_shops d1 +WHERE agg.dim_shops_id = d1.id +and +d1.label in ('foo', 'bar') +GROUP BY d1.label +ORDER BY d1.label +POSTHOOK: type: QUERY +POSTHOOK: Input: default@agg_01 +POSTHOOK: Input: default@agg_01@dim_shops_id=1 +POSTHOOK: Input: default@agg_01@dim_shops_id=2 +POSTHOOK: Input: default@agg_01@dim_shops_id=3 +POSTHOOK: Input: default@dim_shops +#### A masked pattern was here #### +bar 3 15 +foo 3 6 +PREHOOK: query: EXPLAIN +SELECT amount FROM agg_01, dim_shops WHERE dim_shops_id = id AND label = 'foo' +UNION ALL +SELECT amount FROM agg_01, dim_shops WHERE dim_shops_id = id AND label = 'bar' +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +SELECT amount FROM agg_01, dim_shops WHERE dim_shops_id = id AND label = 'foo' +UNION ALL +SELECT amount FROM agg_01, dim_shops WHERE dim_shops_id = id AND label = 'bar' +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-3 depends on stages: Stage-2 + Stage-1 depends on stages: Stage-3 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 2 + Map Operator Tree: + TableScan + alias: dim_shops + filterExpr: (id is not null and (label = 'foo')) (type: boolean) + Statistics: Num rows: 3 Data size: 15 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (id is not null and (label = 'foo')) (type: boolean) + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 dim_shops_id (type: int) + 1 id (type: int) + Select Operator + expressions: id (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: dim_shops_id + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + target column name: dim_shops_id + target work: Map 1 + Local Work: + Map Reduce Local Work + + Stage: Stage-3 + Spark +#### A masked pattern was here #### + Vertices: + Map 4 + Map Operator Tree: + TableScan + alias: dim_shops + filterExpr: (id is not null and (label = 'bar')) (type: boolean) + Statistics: Num rows: 3 Data size: 15 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (id is not null and (label = 'bar')) (type: boolean) + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 dim_shops_id (type: int) + 1 id (type: int) + Select Operator + expressions: id (type: int) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: dim_shops_id + Statistics: Num rows: 1 Data size: 5 Basic stats: COMPLETE Column stats: NONE + target column name: dim_shops_id + target work: Map 3 + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: agg_01 + filterExpr: dim_shops_id is not null (type: boolean) + Statistics: Num rows: 9 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 dim_shops_id (type: int) + 1 id (type: int) + outputColumnNames: _col0, _col1, _col5 + input vertices: + 1 Map 2 + Statistics: Num rows: 9 Data size: 29 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col1 = _col5) (type: boolean) + Statistics: Num rows: 4 Data size: 12 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: decimal(10,0)) + outputColumnNames: _col0 + Statistics: Num rows: 4 Data size: 12 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 8 Data size: 24 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 + Local Work: + Map Reduce Local Work + Map 3 + Map Operator Tree: + TableScan + alias: agg_01 + filterExpr: dim_shops_id is not null (type: boolean) + Statistics: Num rows: 9 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 dim_shops_id (type: int) + 1 id (type: int) + outputColumnNames: _col0, _col1, _col5 + input vertices: + 1 Map 4 + Statistics: Num rows: 9 Data size: 29 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col1 = _col5) (type: boolean) + Statistics: Num rows: 4 Data size: 12 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: decimal(10,0)) + outputColumnNames: _col0 + Statistics: Num rows: 4 Data size: 12 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 8 Data size: 24 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 + Local Work: + Map Reduce Local Work + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: SELECT amount FROM agg_01, dim_shops WHERE dim_shops_id = id AND label = 'foo' +UNION ALL +SELECT amount FROM agg_01, dim_shops WHERE dim_shops_id = id AND label = 'bar' +PREHOOK: type: QUERY +PREHOOK: Input: default@agg_01 +PREHOOK: Input: default@agg_01@dim_shops_id=1 +PREHOOK: Input: default@agg_01@dim_shops_id=2 +PREHOOK: Input: default@agg_01@dim_shops_id=3 +PREHOOK: Input: default@dim_shops +#### A masked pattern was here #### +POSTHOOK: query: SELECT amount FROM agg_01, dim_shops WHERE dim_shops_id = id AND label = 'foo' +UNION ALL +SELECT amount FROM agg_01, dim_shops WHERE dim_shops_id = id AND label = 'bar' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@agg_01 +POSTHOOK: Input: default@agg_01@dim_shops_id=1 +POSTHOOK: Input: default@agg_01@dim_shops_id=2 +POSTHOOK: Input: default@agg_01@dim_shops_id=3 +POSTHOOK: Input: default@dim_shops +#### A masked pattern was here #### +1 +2 +3 +4 +5 +6 diff --git ql/src/test/results/clientpositive/spark/spark_vectorized_dynamic_partition_pruning.q.out ql/src/test/results/clientpositive/spark/spark_vectorized_dynamic_partition_pruning.q.out new file mode 100644 index 0000000..9cab06c --- /dev/null +++ ql/src/test/results/clientpositive/spark/spark_vectorized_dynamic_partition_pruning.q.out @@ -0,0 +1,5822 @@ +PREHOOK: query: select distinct ds from srcpart +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +#### A masked pattern was here #### +POSTHOOK: query: select distinct ds from srcpart +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 #### +2008-04-08 +2008-04-09 +PREHOOK: query: select distinct hr from srcpart +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +#### A masked pattern was here #### +POSTHOOK: query: select distinct hr from srcpart +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 #### +11 +12 +PREHOOK: query: EXPLAIN create table srcpart_date as select ds as ds, ds as `date` from srcpart group by ds +PREHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: query: EXPLAIN create table srcpart_date as select ds as ds, ds as `date` from srcpart group by ds +POSTHOOK: type: CREATETABLE_AS_SELECT +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + Stage-3 depends on stages: Stage-0 + Stage-2 depends on stages: Stage-3 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 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: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + 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 + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcpart_date + Execution mode: vectorized + + Stage: Stage-0 + Move Operator + files: + hdfs directory: true +#### A masked pattern was here #### + + Stage: Stage-3 + Create Table Operator: + Create Table + columns: ds string, date string + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat + serde name: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.srcpart_date + + Stage: Stage-2 + Stats-Aggr Operator + +PREHOOK: query: create table srcpart_date stored as orc as select ds as ds, ds as `date` from srcpart group by ds +PREHOOK: type: CREATETABLE_AS_SELECT +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Output: database:default +PREHOOK: Output: default@srcpart_date +POSTHOOK: query: create table srcpart_date stored as orc as select ds as ds, ds as `date` from srcpart group by ds +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Output: database:default +POSTHOOK: Output: default@srcpart_date +PREHOOK: query: create table srcpart_hour stored as orc as select hr as hr, hr as hour from srcpart group by hr +PREHOOK: type: CREATETABLE_AS_SELECT +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Output: database:default +PREHOOK: Output: default@srcpart_hour +POSTHOOK: query: create table srcpart_hour stored as orc as select hr as hr, hr as hour from srcpart group by hr +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Output: database:default +POSTHOOK: Output: default@srcpart_hour +PREHOOK: query: create table srcpart_date_hour stored as orc as select ds as ds, ds as `date`, hr as hr, hr as hour from srcpart group by ds, hr +PREHOOK: type: CREATETABLE_AS_SELECT +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Output: database:default +PREHOOK: Output: default@srcpart_date_hour +POSTHOOK: query: create table srcpart_date_hour stored as orc as select ds as ds, ds as `date`, hr as hr, hr as hour from srcpart group by ds, hr +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Output: database:default +POSTHOOK: Output: default@srcpart_date_hour +PREHOOK: query: create table srcpart_double_hour stored as orc as select (hr*2) as hr, hr as hour from srcpart group by hr +PREHOOK: type: CREATETABLE_AS_SELECT +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Output: database:default +PREHOOK: Output: default@srcpart_double_hour +POSTHOOK: query: create table srcpart_double_hour stored as orc as select (hr*2) as hr, hr as hour from srcpart group by hr +POSTHOOK: type: CREATETABLE_AS_SELECT +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Output: database:default +POSTHOOK: Output: default@srcpart_double_hour +PREHOOK: query: -- single column, single key +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08' +PREHOOK: type: QUERY +POSTHOOK: query: -- single column, single key +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08' +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 5 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + Execution mode: vectorized + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: ds is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Execution mode: vectorized + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE 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 3 + 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 + 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 + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08' +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_date +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_date +#### A masked pattern was here #### +1000 +PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08' +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08' +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 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: ds is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Execution mode: vectorized + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE 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 3 + 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 + 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 + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08' +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_date +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_date +#### A masked pattern was here #### +1000 +PREHOOK: query: select count(*) from srcpart where ds = '2008-04-08' +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where ds = '2008-04-08' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +#### A masked pattern was here #### +1000 +PREHOOK: query: -- multiple sources, single key +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.`date` = '2008-04-08' and srcpart_hour.hour = 11 +PREHOOK: type: QUERY +POSTHOOK: query: -- multiple sources, single key +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.`date` = '2008-04-08' and srcpart_hour.hour = 11 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 7 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + Execution mode: vectorized + Map 8 + Map Operator Tree: + TableScan + alias: srcpart_hour + filterExpr: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 344 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 172 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: hr (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 172 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 172 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: hr + Statistics: Num rows: 1 Data size: 172 Basic stats: COMPLETE Column stats: NONE + target column name: hr + target work: Map 1 + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + value expressions: hr (type: string) + Map 5 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Execution mode: vectorized + Map 6 + Map Operator Tree: + TableScan + alias: srcpart_hour + filterExpr: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 344 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 172 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: hr (type: string) + sort order: + + Map-reduce partition columns: hr (type: string) + Statistics: Num rows: 1 Data size: 172 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + outputColumnNames: _col3 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col3 (type: string) + sort order: + + Map-reduce partition columns: _col3 (type: string) + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col3 (type: string) + 1 hr (type: string) + Statistics: Num rows: 2420 Data size: 25709 Basic stats: COMPLETE 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 4 + 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 + 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 + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.`date` = '2008-04-08' and srcpart_hour.hour = 11 +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_date +PREHOOK: Input: default@srcpart_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.`date` = '2008-04-08' and srcpart_hour.hour = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_date +POSTHOOK: Input: default@srcpart_hour +#### A masked pattern was here #### +500 +PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.`date` = '2008-04-08' and srcpart_hour.hour = 11 +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.`date` = '2008-04-08' and srcpart_hour.hour = 11 +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 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: (ds is not null and hr is not null) (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + value expressions: hr (type: string) + Map 5 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Execution mode: vectorized + Map 6 + Map Operator Tree: + TableScan + alias: srcpart_hour + filterExpr: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 344 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 172 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: hr (type: string) + sort order: + + Map-reduce partition columns: hr (type: string) + Statistics: Num rows: 1 Data size: 172 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + outputColumnNames: _col3 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col3 (type: string) + sort order: + + Map-reduce partition columns: _col3 (type: string) + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col3 (type: string) + 1 hr (type: string) + Statistics: Num rows: 2420 Data size: 25709 Basic stats: COMPLETE 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 4 + 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 + 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 + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.`date` = '2008-04-08' and srcpart_hour.hour = 11 +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_date +PREHOOK: Input: default@srcpart_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.`date` = '2008-04-08' and srcpart_hour.hour = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_date +POSTHOOK: Input: default@srcpart_hour +#### A masked pattern was here #### +500 +PREHOOK: query: select count(*) from srcpart where hr = 11 and ds = '2008-04-08' +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where hr = 11 and ds = '2008-04-08' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +500 +PREHOOK: query: -- multiple columns single source +EXPLAIN select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11 +PREHOOK: type: QUERY +POSTHOOK: query: -- multiple columns single source +EXPLAIN select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 5 + Map Operator Tree: + TableScan + alias: srcpart_date_hour + filterExpr: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 4 Data size: 1440 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + Map 6 + Map Operator Tree: + TableScan + alias: srcpart_date_hour + filterExpr: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 4 Data size: 1440 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: hr (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: hr + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + target column name: hr + target work: Map 1 + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string), hr (type: string) + sort order: ++ + Map-reduce partition columns: ds (type: string), hr (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date_hour + filterExpr: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 4 Data size: 1440 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string), hr (type: string) + sort order: ++ + Map-reduce partition columns: ds (type: string), hr (type: string) + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 ds (type: string), hr (type: string) + 1 ds (type: string), hr (type: string) + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE 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 3 + 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 + 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 + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11 +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_date_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_date_hour +#### A masked pattern was here #### +500 +PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11 +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11 +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 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: (ds is not null and hr is not null) (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string), hr (type: string) + sort order: ++ + Map-reduce partition columns: ds (type: string), hr (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date_hour + filterExpr: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 4 Data size: 1440 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string), hr (type: string) + sort order: ++ + Map-reduce partition columns: ds (type: string), hr (type: string) + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 ds (type: string), hr (type: string) + 1 ds (type: string), hr (type: string) + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE 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 3 + 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 + 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 + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11 +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_date_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_date_hour +#### A masked pattern was here #### +500 +PREHOOK: query: select count(*) from srcpart where ds = '2008-04-08' and hr = 11 +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where ds = '2008-04-08' and hr = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +500 +PREHOOK: query: -- empty set +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = 'I DONT EXIST' +PREHOOK: type: QUERY +POSTHOOK: query: -- empty set +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = 'I DONT EXIST' +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 5 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = 'I DONT EXIST')) (type: boolean) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = 'I DONT EXIST')) (type: boolean) + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + Execution mode: vectorized + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: ds is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = 'I DONT EXIST')) (type: boolean) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = 'I DONT EXIST')) (type: boolean) + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Execution mode: vectorized + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE 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 3 + 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 + 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 + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = 'I DONT EXIST' +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_date +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = 'I DONT EXIST' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_date +#### A masked pattern was here #### +0 +PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = 'I DONT EXIST' +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = 'I DONT EXIST' +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 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: ds is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = 'I DONT EXIST')) (type: boolean) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = 'I DONT EXIST')) (type: boolean) + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Execution mode: vectorized + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE 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 3 + 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 + 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 + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = 'I DONT EXIST' +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_date +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = 'I DONT EXIST' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_date +#### A masked pattern was here #### +0 +PREHOOK: query: select count(*) from srcpart where ds = 'I DONT EXIST' +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where ds = 'I DONT EXIST' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +#### A masked pattern was here #### +0 +PREHOOK: query: -- expressions +EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11 +PREHOOK: type: QUERY +POSTHOOK: query: -- expressions +EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 5 + Map Operator Tree: + TableScan + alias: srcpart_double_hour + filterExpr: (UDFToDouble(UDFToInteger((hr / 2))) is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (UDFToDouble(UDFToInteger((hr / 2))) is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: UDFToDouble(UDFToInteger((hr / 2))) (type: double) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: double) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: UDFToDouble(hr) + Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE + target column name: hr + target work: Map 1 + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: UDFToDouble(hr) is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: UDFToDouble(hr) is not null (type: boolean) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: UDFToDouble(hr) (type: double) + sort order: + + Map-reduce partition columns: UDFToDouble(hr) (type: double) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_double_hour + filterExpr: (UDFToDouble(UDFToInteger((hr / 2))) is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (UDFToDouble(UDFToInteger((hr / 2))) is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: UDFToDouble(UDFToInteger((hr / 2))) (type: double) + sort order: + + Map-reduce partition columns: UDFToDouble(UDFToInteger((hr / 2))) (type: double) + Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 UDFToDouble(hr) (type: double) + 1 UDFToDouble(UDFToInteger((hr / 2))) (type: double) + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE 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 3 + 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 + 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 + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11 +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_double_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_double_hour +#### A masked pattern was here #### +1000 +PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11 +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 5 + Map Operator Tree: + TableScan + alias: srcpart_double_hour + filterExpr: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: hr (type: double) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: double) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: (hr * 2) + Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE + target column name: hr + target work: Map 1 + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: (hr * 2) is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (hr * 2) is not null (type: boolean) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: (hr * 2) (type: double) + sort order: + + Map-reduce partition columns: (hr * 2) (type: double) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_double_hour + filterExpr: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: hr (type: double) + sort order: + + Map-reduce partition columns: hr (type: double) + Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 (hr * 2) (type: double) + 1 hr (type: double) + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE 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 3 + 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 + 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 + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11 +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_double_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_double_hour +#### A masked pattern was here #### +1000 +PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11 +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11 +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 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: UDFToDouble(hr) is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: UDFToDouble(hr) is not null (type: boolean) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: UDFToDouble(hr) (type: double) + sort order: + + Map-reduce partition columns: UDFToDouble(hr) (type: double) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_double_hour + filterExpr: (UDFToDouble(UDFToInteger((hr / 2))) is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (UDFToDouble(UDFToInteger((hr / 2))) is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: UDFToDouble(UDFToInteger((hr / 2))) (type: double) + sort order: + + Map-reduce partition columns: UDFToDouble(UDFToInteger((hr / 2))) (type: double) + Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 UDFToDouble(hr) (type: double) + 1 UDFToDouble(UDFToInteger((hr / 2))) (type: double) + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE 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 3 + 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 + 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 + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11 +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_double_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_double_hour +#### A masked pattern was here #### +1000 +PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11 +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11 +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 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: (hr * 2) is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (hr * 2) is not null (type: boolean) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: (hr * 2) (type: double) + sort order: + + Map-reduce partition columns: (hr * 2) (type: double) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_double_hour + filterExpr: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: hr (type: double) + sort order: + + Map-reduce partition columns: hr (type: double) + Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 (hr * 2) (type: double) + 1 hr (type: double) + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE 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 3 + 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 + 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 + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11 +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_double_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_double_hour +#### A masked pattern was here #### +1000 +PREHOOK: query: select count(*) from srcpart where hr = 11 +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where hr = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 +#### A masked pattern was here #### +1000 +PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_double_hour on (cast(srcpart.hr*2 as string) = cast(srcpart_double_hour.hr as string)) where srcpart_double_hour.hour = 11 +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_double_hour on (cast(srcpart.hr*2 as string) = cast(srcpart_double_hour.hr as string)) where srcpart_double_hour.hour = 11 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 5 + Map Operator Tree: + TableScan + alias: srcpart_double_hour + filterExpr: (UDFToString(hr) is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (UDFToString(hr) is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: UDFToString(hr) (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: UDFToString((hr * 2)) + Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE + target column name: hr + target work: Map 1 + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: UDFToString((hr * 2)) is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: UDFToString((hr * 2)) is not null (type: boolean) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: UDFToString((hr * 2)) (type: string) + sort order: + + Map-reduce partition columns: UDFToString((hr * 2)) (type: string) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_double_hour + filterExpr: (UDFToString(hr) is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (UDFToString(hr) is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: UDFToString(hr) (type: string) + sort order: + + Map-reduce partition columns: UDFToString(hr) (type: string) + Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 UDFToString((hr * 2)) (type: string) + 1 UDFToString(hr) (type: string) + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE 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 3 + 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 + 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 + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_double_hour on (cast(srcpart.hr*2 as string) = cast(srcpart_double_hour.hr as string)) where srcpart_double_hour.hour = 11 +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_double_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_double_hour on (cast(srcpart.hr*2 as string) = cast(srcpart_double_hour.hr as string)) where srcpart_double_hour.hour = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_double_hour +#### A masked pattern was here #### +1000 +PREHOOK: query: select count(*) from srcpart where cast(hr as string) = 11 +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where cast(hr as string) = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 +#### A masked pattern was here #### +1000 +Warning: Shuffle Join JOIN[13][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: -- parent is reduce tasks +EXPLAIN select count(*) from srcpart join (select ds as ds, ds as `date` from srcpart group by ds) s on (srcpart.ds = s.ds) where s.`date` = '2008-04-08' +PREHOOK: type: QUERY +POSTHOOK: query: -- parent is reduce tasks +EXPLAIN select count(*) from srcpart join (select ds as ds, ds as `date` from srcpart group by ds) s on (srcpart.ds = s.ds) where s.`date` = '2008-04-08' +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 5 <- Map 4 (GROUP, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 5 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: (ds = '2008-04-08') (type: boolean) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: (ds = '2008-04-08') (type: boolean) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: '2008-04-08' (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1000 Data size: 10624 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: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE 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 3 + 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 + 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 + Execution mode: vectorized + Reducer 5 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[13][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +PREHOOK: query: select count(*) from srcpart join (select ds as ds, ds as `date` from srcpart group by ds) s on (srcpart.ds = s.ds) where s.`date` = '2008-04-08' +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join (select ds as ds, ds as `date` from srcpart group by ds) s on (srcpart.ds = s.ds) where s.`date` = '2008-04-08' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +#### A masked pattern was here #### +1000 +PREHOOK: query: select count(*) from srcpart where ds = '2008-04-08' +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where ds = '2008-04-08' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +#### A masked pattern was here #### +1000 +Warning: Shuffle Join JOIN[4][tables = [srcpart, srcpart_date_hour]] in Work 'Reducer 2' is a cross product +PREHOOK: query: -- non-equi join +EXPLAIN select count(*) from srcpart, srcpart_date_hour where (srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11) and (srcpart.ds = srcpart_date_hour.ds or srcpart.hr = srcpart_date_hour.hr) +PREHOOK: type: QUERY +POSTHOOK: query: -- non-equi join +EXPLAIN select count(*) from srcpart, srcpart_date_hour where (srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11) and (srcpart.ds = srcpart_date_hour.ds or srcpart.hr = srcpart_date_hour.hr) +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 (PARTITION-LEVEL SORT, 1), Map 4 (PARTITION-LEVEL SORT, 1) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + value expressions: ds (type: string), hr (type: string) + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date_hour + filterExpr: ((date = '2008-04-08') and (hour = 11)) (type: boolean) + Statistics: Num rows: 4 Data size: 1440 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((date = '2008-04-08') and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + value expressions: ds (type: string), hr (type: string) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col2, _col3, _col7, _col9 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col2 = _col7) or (_col3 = _col9)) (type: boolean) + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE 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 3 + 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 + 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 + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Shuffle Join JOIN[4][tables = [srcpart, srcpart_date_hour]] in Work 'Reducer 2' is a cross product +PREHOOK: query: select count(*) from srcpart, srcpart_date_hour where (srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11) and (srcpart.ds = srcpart_date_hour.ds or srcpart.hr = srcpart_date_hour.hr) +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_date_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart, srcpart_date_hour where (srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11) and (srcpart.ds = srcpart_date_hour.ds or srcpart.hr = srcpart_date_hour.hr) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_date_hour +#### A masked pattern was here #### +1500 +PREHOOK: query: -- old style join syntax +EXPLAIN select count(*) from srcpart, srcpart_date_hour where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11 and srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr +PREHOOK: type: QUERY +POSTHOOK: query: -- old style join syntax +EXPLAIN select count(*) from srcpart, srcpart_date_hour where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11 and srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 5 + Map Operator Tree: + TableScan + alias: srcpart_date_hour + filterExpr: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 4 Data size: 1440 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + Map 6 + Map Operator Tree: + TableScan + alias: srcpart_date_hour + filterExpr: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 4 Data size: 1440 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: hr (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: hr + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + target column name: hr + target work: Map 1 + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string), hr (type: string) + sort order: ++ + Map-reduce partition columns: ds (type: string), hr (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date_hour + filterExpr: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 4 Data size: 1440 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string), hr (type: string) + sort order: ++ + Map-reduce partition columns: ds (type: string), hr (type: string) + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 ds (type: string), hr (type: string) + 1 ds (type: string), hr (type: string) + outputColumnNames: _col2, _col3, _col7, _col9 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col2 = _col7) and (_col3 = _col9)) (type: boolean) + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE 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 3 + 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 + 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 + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart, srcpart_date_hour where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11 and srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_date_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart, srcpart_date_hour where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11 and srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_date_hour +#### A masked pattern was here #### +500 +PREHOOK: query: -- left join +EXPLAIN select count(*) from srcpart left join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08' +PREHOOK: type: QUERY +POSTHOOK: query: -- left join +EXPLAIN select count(*) from srcpart left join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08' +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 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + value expressions: date (type: string) + Execution mode: vectorized + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + outputColumnNames: _col8 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col8 = '2008-04-08') (type: boolean) + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE 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 3 + 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 + 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 + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select count(*) from srcpart_date left join srcpart on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08' +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart_date left join srcpart on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08' +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 5 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (date = '2008-04-08') (type: boolean) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (date = '2008-04-08') (type: boolean) + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 4 + Execution mode: vectorized + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (date = '2008-04-08') (type: boolean) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (date = '2008-04-08') (type: boolean) + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Execution mode: vectorized + Map 4 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Outer Join0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE 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 3 + 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 + 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 + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: -- full outer +EXPLAIN select count(*) from srcpart full outer join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08' +PREHOOK: type: QUERY +POSTHOOK: query: -- full outer +EXPLAIN select count(*) from srcpart full outer join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08' +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 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + value expressions: date (type: string) + Execution mode: vectorized + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Outer Join 0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + outputColumnNames: _col8 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col8 = '2008-04-08') (type: boolean) + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE 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 3 + 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 + 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 + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: -- with static pruning +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.`date` = '2008-04-08' and srcpart_hour.hour = 11 and srcpart.hr = 11 +PREHOOK: type: QUERY +POSTHOOK: query: -- with static pruning +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.`date` = '2008-04-08' and srcpart_hour.hour = 11 and srcpart.hr = 11 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 7 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + Execution mode: vectorized + Map 8 + Map Operator Tree: + TableScan + alias: srcpart_hour + filterExpr: ((hr is not null and (hour = 11)) and (hr = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 344 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((hr is not null and (hour = 11)) and (hr = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 172 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: '11' (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 172 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 172 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: hr + Statistics: Num rows: 1 Data size: 172 Basic stats: COMPLETE Column stats: NONE + target column name: hr + target work: Map 1 + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: (hr = 11) (type: boolean) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Map 5 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Execution mode: vectorized + Map 6 + Map Operator Tree: + TableScan + alias: srcpart_hour + filterExpr: ((hr is not null and (hour = 11)) and (hr = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 344 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((hr is not null and (hour = 11)) and (hr = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 172 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: '11' (type: string) + sort order: + + Map-reduce partition columns: '11' (type: string) + Statistics: Num rows: 1 Data size: 172 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: '11' (type: string) + sort order: + + Map-reduce partition columns: '11' (type: string) + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col3 (type: string) + 1 hr (type: string) + Statistics: Num rows: 1210 Data size: 12854 Basic stats: COMPLETE 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 4 + 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 + 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 + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.`date` = '2008-04-08' and srcpart_hour.hour = 11 and srcpart.hr = 11 +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart_date +PREHOOK: Input: default@srcpart_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.`date` = '2008-04-08' and srcpart_hour.hour = 11 and srcpart.hr = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 +POSTHOOK: Input: default@srcpart_date +POSTHOOK: Input: default@srcpart_hour +#### A masked pattern was here #### +500 +PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.`date` = '2008-04-08' and srcpart.hr = 13 +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.`date` = '2008-04-08' and srcpart.hr = 13 +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 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2) + Reducer 4 <- Reducer 3 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: ((ds is not null and hr is not null) and (hr = 13)) (type: boolean) + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Filter Operator + predicate: ((ds is not null and hr is not null) and (hr = 13)) (type: boolean) + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Map 5 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Execution mode: vectorized + Map 6 + Map Operator Tree: + TableScan + alias: srcpart_hour + filterExpr: (hr = 13) (type: boolean) + Statistics: Num rows: 2 Data size: 344 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (hr = 13) (type: boolean) + Statistics: Num rows: 1 Data size: 172 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: '13' (type: string) + sort order: + + Map-reduce partition columns: '13' (type: string) + Statistics: Num rows: 1 Data size: 172 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: '13' (type: string) + sort order: + + Map-reduce partition columns: '13' (type: string) + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col3 (type: string) + 1 hr (type: string) + Statistics: Num rows: 1 Data size: 0 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 4 + 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 + 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 + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.`date` = '2008-04-08' and srcpart.hr = 13 +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart_date +PREHOOK: Input: default@srcpart_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.`date` = '2008-04-08' and srcpart.hr = 13 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart_date +POSTHOOK: Input: default@srcpart_hour +#### A masked pattern was here #### +0 +PREHOOK: query: -- union + subquery +EXPLAIN select count(*) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +PREHOOK: type: QUERY +POSTHOOK: query: -- union + subquery +EXPLAIN select count(*) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark + Edges: + Reducer 11 <- Map 10 (GROUP, 1) + Reducer 9 <- Map 8 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 10 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(_col0) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Map 8 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(_col0) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Reducer 11 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + Reducer 9 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + + Stage: Stage-1 + Spark + Edges: + Reducer 5 <- Map 4 (GROUP, 1) + Reducer 7 <- Map 6 (GROUP, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2), Reducer 7 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: ds is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 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: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(_col0) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Map 6 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(_col0) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE 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 3 + 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 + 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 + Execution mode: vectorized + Reducer 5 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 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: 168 Basic stats: COMPLETE Column stats: NONE + Reducer 7 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 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: 168 Basic stats: COMPLETE Column stats: NONE + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +PREHOOK: query: EXPLAIN select distinct(ds) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select distinct(ds) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark + Edges: + Reducer 11 <- Map 10 (GROUP, 1) + Reducer 9 <- Map 8 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 10 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(_col0) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Map 8 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(_col0) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Reducer 11 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + Reducer 9 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + + Stage: Stage-1 + Spark + Edges: + Reducer 5 <- Map 4 (GROUP, 1) + Reducer 7 <- Map 6 (GROUP, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2), Reducer 7 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: ds is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 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: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(_col0) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Map 6 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(_col0) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2200 Data size: 23372 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: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Reducer 3 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1100 Data size: 11686 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 + Execution mode: vectorized + Reducer 5 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 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: 168 Basic stats: COMPLETE Column stats: NONE + Reducer 7 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 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: 168 Basic stats: COMPLETE Column stats: NONE + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select distinct(ds) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +#### A masked pattern was here #### +POSTHOOK: query: select distinct(ds) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 #### +2008-04-08 +2008-04-09 +PREHOOK: query: EXPLAIN select ds from (select distinct(ds) as ds from srcpart union all select distinct(ds) as ds from srcpart) s where s.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select ds from (select distinct(ds) as ds from srcpart union all select distinct(ds) as ds from srcpart) s where s.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark + Edges: + Reducer 11 <- Map 10 (GROUP, 1) + Reducer 13 <- Map 12 (GROUP, 1) + Reducer 15 <- Map 10 (GROUP, 1) + Reducer 17 <- Map 12 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 10 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(_col0) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Map 12 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(_col0) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Reducer 11 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + Reducer 13 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + Reducer 15 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 4 + Reducer 17 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 4 + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 5 <- Map 1 (GROUP, 2) + Reducer 7 <- Map 6 (GROUP, 1) + Reducer 9 <- Map 8 (GROUP, 1) + Reducer 3 <- Reducer 2 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2), Reducer 7 (PARTITION-LEVEL SORT, 2), Reducer 9 (PARTITION-LEVEL SORT, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: ds is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 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: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map 6 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(_col0) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Map 8 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(_col0) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Reducer 2 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1000 Data size: 10624 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: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Execution mode: vectorized + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 2200 Data size: 23372 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 + Reducer 5 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1000 Data size: 10624 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: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Execution mode: vectorized + Reducer 7 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 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: 168 Basic stats: COMPLETE Column stats: NONE + Reducer 9 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 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: 168 Basic stats: COMPLETE Column stats: NONE + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select ds from (select distinct(ds) as ds from srcpart union all select distinct(ds) as ds from srcpart) s where s.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +#### A masked pattern was here #### +POSTHOOK: query: select ds from (select distinct(ds) as ds from srcpart union all select distinct(ds) as ds from srcpart) s where s.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 #### +2008-04-08 +2008-04-08 +2008-04-09 +2008-04-09 +PREHOOK: query: -- single column, single key +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08' +PREHOOK: type: QUERY +POSTHOOK: query: -- single column, single key +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08' +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 3 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 ds (type: string) + 1 ds (type: string) + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + Local Work: + Map Reduce Local Work + Execution mode: vectorized + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: ds is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + input vertices: + 1 Map 3 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE 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) + Local Work: + Map Reduce Local Work + 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 + 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 + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08' +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_date +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_date +#### A masked pattern was here #### +1000 +PREHOOK: query: select count(*) from srcpart where ds = '2008-04-08' +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where ds = '2008-04-08' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +#### A masked pattern was here #### +1000 +PREHOOK: query: -- multiple sources, single key +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.`date` = '2008-04-08' and srcpart_hour.hour = 11 +PREHOOK: type: QUERY +POSTHOOK: query: -- multiple sources, single key +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.`date` = '2008-04-08' and srcpart_hour.hour = 11 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 3 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 ds (type: string) + 1 ds (type: string) + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + Local Work: + Map Reduce Local Work + Execution mode: vectorized + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_hour + filterExpr: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 344 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 172 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 _col3 (type: string) + 1 hr (type: string) + Select Operator + expressions: hr (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 172 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 172 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: hr + Statistics: Num rows: 1 Data size: 172 Basic stats: COMPLETE Column stats: NONE + target column name: hr + target work: Map 1 + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + outputColumnNames: _col3 + input vertices: + 1 Map 3 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col3 (type: string) + 1 hr (type: string) + input vertices: + 1 Map 4 + Statistics: Num rows: 2420 Data size: 25709 Basic stats: COMPLETE 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) + Local Work: + Map Reduce Local Work + 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 + 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 + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.`date` = '2008-04-08' and srcpart_hour.hour = 11 +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_date +PREHOOK: Input: default@srcpart_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.`date` = '2008-04-08' and srcpart_hour.hour = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_date +POSTHOOK: Input: default@srcpart_hour +#### A masked pattern was here #### +500 +PREHOOK: query: select count(*) from srcpart where hr = 11 and ds = '2008-04-08' +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where hr = 11 and ds = '2008-04-08' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +500 +PREHOOK: query: -- multiple columns single source +EXPLAIN select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11 +PREHOOK: type: QUERY +POSTHOOK: query: -- multiple columns single source +EXPLAIN select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 3 + Map Operator Tree: + TableScan + alias: srcpart_date_hour + filterExpr: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 4 Data size: 1440 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 ds (type: string), hr (type: string) + 1 ds (type: string), hr (type: string) + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + Select Operator + expressions: hr (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: hr + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + target column name: hr + target work: Map 1 + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 ds (type: string), hr (type: string) + 1 ds (type: string), hr (type: string) + input vertices: + 1 Map 3 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE 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) + Local Work: + Map Reduce Local Work + 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 + 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 + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11 +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_date_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_date_hour +#### A masked pattern was here #### +500 +PREHOOK: query: select count(*) from srcpart where ds = '2008-04-08' and hr = 11 +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where ds = '2008-04-08' and hr = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +#### A masked pattern was here #### +500 +PREHOOK: query: -- empty set +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = 'I DONT EXIST' +PREHOOK: type: QUERY +POSTHOOK: query: -- empty set +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = 'I DONT EXIST' +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 3 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = 'I DONT EXIST')) (type: boolean) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = 'I DONT EXIST')) (type: boolean) + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 ds (type: string) + 1 ds (type: string) + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + Local Work: + Map Reduce Local Work + Execution mode: vectorized + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: ds is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + input vertices: + 1 Map 3 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE 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) + Local Work: + Map Reduce Local Work + 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 + 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 + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: -- Disabled until TEZ-1486 is fixed +-- select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = 'I DONT EXIST'; + +-- expressions +EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11 +PREHOOK: type: QUERY +POSTHOOK: query: -- Disabled until TEZ-1486 is fixed +-- select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = 'I DONT EXIST'; + +-- expressions +EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 3 + Map Operator Tree: + TableScan + alias: srcpart_double_hour + filterExpr: (UDFToDouble(UDFToInteger((hr / 2))) is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (UDFToDouble(UDFToInteger((hr / 2))) is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 UDFToDouble(hr) (type: double) + 1 UDFToDouble(UDFToInteger((hr / 2))) (type: double) + Select Operator + expressions: UDFToDouble(UDFToInteger((hr / 2))) (type: double) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: double) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: UDFToDouble(hr) + Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE + target column name: hr + target work: Map 1 + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: UDFToDouble(hr) is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: UDFToDouble(hr) is not null (type: boolean) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 UDFToDouble(hr) (type: double) + 1 UDFToDouble(UDFToInteger((hr / 2))) (type: double) + input vertices: + 1 Map 3 + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE 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) + Local Work: + Map Reduce Local Work + 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 + 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 + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11 +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_double_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_double_hour +#### A masked pattern was here #### +1000 +PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11 +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 3 + Map Operator Tree: + TableScan + alias: srcpart_double_hour + filterExpr: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (hr is not null and (hour = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 (hr * 2) (type: double) + 1 hr (type: double) + Select Operator + expressions: hr (type: double) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: double) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: (hr * 2) + Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE + target column name: hr + target work: Map 1 + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: (hr * 2) is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (hr * 2) is not null (type: boolean) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 (hr * 2) (type: double) + 1 hr (type: double) + input vertices: + 1 Map 3 + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE 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) + Local Work: + Map Reduce Local Work + 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 + 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 + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11 +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Input: default@srcpart_double_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Input: default@srcpart_double_hour +#### A masked pattern was here #### +1000 +PREHOOK: query: select count(*) from srcpart where hr = 11 +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where hr = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 +#### A masked pattern was here #### +1000 +Warning: Map Join MAPJOIN[23][bigTable=?] in task 'Stage-1:MAPRED' is a cross product +PREHOOK: query: -- parent is reduce tasks +EXPLAIN select count(*) from srcpart join (select ds as ds, ds as `date` from srcpart group by ds) s on (srcpart.ds = s.ds) where s.`date` = '2008-04-08' +PREHOOK: type: QUERY +POSTHOOK: query: -- parent is reduce tasks +EXPLAIN select count(*) from srcpart join (select ds as ds, ds as `date` from srcpart group by ds) s on (srcpart.ds = s.ds) where s.`date` = '2008-04-08' +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark + Edges: + Reducer 4 <- Map 3 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 3 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: (ds = '2008-04-08') (type: boolean) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: '2008-04-08' (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1000 Data size: 10624 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: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Reducer 4 + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 + 1 + Execution mode: vectorized + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: (ds = '2008-04-08') (type: boolean) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + input vertices: + 1 Reducer 4 + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE 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) + Local Work: + Map Reduce Local Work + 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 + 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 + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +Warning: Map Join MAPJOIN[23][bigTable=?] in task 'Stage-1:MAPRED' is a cross product +PREHOOK: query: select count(*) from srcpart join (select ds as ds, ds as `date` from srcpart group by ds) s on (srcpart.ds = s.ds) where s.`date` = '2008-04-08' +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join (select ds as ds, ds as `date` from srcpart group by ds) s on (srcpart.ds = s.ds) where s.`date` = '2008-04-08' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +#### A masked pattern was here #### +1000 +PREHOOK: query: select count(*) from srcpart where ds = '2008-04-08' +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where ds = '2008-04-08' +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +#### A masked pattern was here #### +1000 +PREHOOK: query: -- left join +EXPLAIN select count(*) from srcpart left join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08' +PREHOOK: type: QUERY +POSTHOOK: query: -- left join +EXPLAIN select count(*) from srcpart left join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08' +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 3 + Map Operator Tree: + TableScan + alias: srcpart_date + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 ds (type: string) + 1 ds (type: string) + Local Work: + Map Reduce Local Work + Execution mode: vectorized + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Left Outer Join0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + outputColumnNames: _col8 + input vertices: + 1 Map 3 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col8 = '2008-04-08') (type: boolean) + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE 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) + Local Work: + Map Reduce Local Work + 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 + 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 + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: EXPLAIN select count(*) from srcpart_date left join srcpart on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08' +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart_date left join srcpart on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08' +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-3 depends on stages: Stage-2 + Stage-1 depends on stages: Stage-3 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (date = '2008-04-08') (type: boolean) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (date = '2008-04-08') (type: boolean) + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 3 + Execution mode: vectorized + + Stage: Stage-3 + Spark +#### A masked pattern was here #### + Vertices: + Map 3 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 ds (type: string) + 1 ds (type: string) + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (date = '2008-04-08') (type: boolean) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (date = '2008-04-08') (type: boolean) + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Left Outer Join0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + input vertices: + 1 Map 3 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE 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) + Local Work: + Map Reduce Local Work + Execution mode: vectorized + 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 + 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 + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: -- full outer +EXPLAIN select count(*) from srcpart full outer join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08' +PREHOOK: type: QUERY +POSTHOOK: query: -- full outer +EXPLAIN select count(*) from srcpart full outer join srcpart_date on (srcpart.ds = srcpart_date.ds) where srcpart_date.`date` = '2008-04-08' +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 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_date + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + value expressions: date (type: string) + Execution mode: vectorized + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Outer Join 0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + outputColumnNames: _col8 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col8 = '2008-04-08') (type: boolean) + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE 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 3 + 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 + 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 + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: -- with static pruning +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.`date` = '2008-04-08' and srcpart_hour.hour = 11 and srcpart.hr = 11 +PREHOOK: type: QUERY +POSTHOOK: query: -- with static pruning +EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.`date` = '2008-04-08' and srcpart_hour.hour = 11 and srcpart.hr = 11 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 3 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 ds (type: string) + 1 ds (type: string) + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + Local Work: + Map Reduce Local Work + Execution mode: vectorized + Map 4 + Map Operator Tree: + TableScan + alias: srcpart_hour + filterExpr: ((hr is not null and (hour = 11)) and (hr = 11)) (type: boolean) + Statistics: Num rows: 2 Data size: 344 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((hr is not null and (hour = 11)) and (hr = 11)) (type: boolean) + Statistics: Num rows: 1 Data size: 172 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 '11' (type: string) + 1 '11' (type: string) + Select Operator + expressions: '11' (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 172 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 172 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: hr + Statistics: Num rows: 1 Data size: 172 Basic stats: COMPLETE Column stats: NONE + target column name: hr + target work: Map 1 + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: (hr = 11) (type: boolean) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + input vertices: + 1 Map 3 + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 '11' (type: string) + 1 '11' (type: string) + input vertices: + 1 Map 4 + Statistics: Num rows: 1210 Data size: 12854 Basic stats: COMPLETE 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) + Local Work: + Map Reduce Local Work + 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 + 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 + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.`date` = '2008-04-08' and srcpart_hour.hour = 11 and srcpart.hr = 11 +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart_date +PREHOOK: Input: default@srcpart_hour +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.`date` = '2008-04-08' and srcpart_hour.hour = 11 and srcpart.hr = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 +POSTHOOK: Input: default@srcpart_date +POSTHOOK: Input: default@srcpart_hour +#### A masked pattern was here #### +500 +PREHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.`date` = '2008-04-08' and srcpart.hr = 13 +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +where srcpart_date.`date` = '2008-04-08' and srcpart.hr = 13 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-3 is a root stage + Stage-2 depends on stages: Stage-3 + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-3 + Spark +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: ((ds is not null and hr is not null) and (hr = 13)) (type: boolean) + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Filter Operator + predicate: ((ds is not null and hr is not null) and (hr = 13)) (type: boolean) + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 ds (type: string) + 1 ds (type: string) + Local Work: + Map Reduce Local Work + + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 2 + Map Operator Tree: + TableScan + alias: srcpart_date + filterExpr: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 2 Data size: 376 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (ds is not null and (date = '2008-04-08')) (type: boolean) + Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 ds (type: string) + 1 ds (type: string) + input vertices: + 0 Map 1 + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 '13' (type: string) + 1 '13' (type: string) + Local Work: + Map Reduce Local Work + Execution mode: vectorized + + Stage: Stage-1 + Spark + Edges: + Reducer 4 <- Map 3 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 3 + Map Operator Tree: + TableScan + alias: srcpart_hour + filterExpr: (hr = 13) (type: boolean) + Statistics: Num rows: 2 Data size: 344 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (hr = 13) (type: boolean) + Statistics: Num rows: 1 Data size: 172 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 '13' (type: string) + 1 '13' (type: string) + input vertices: + 0 Map 2 + Statistics: Num rows: 1 Data size: 0 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) + Local Work: + Map Reduce Local Work + Reducer 4 + 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 + 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 + Execution mode: vectorized + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: -- Disabled until TEZ-1486 is fixed +-- select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +-- where srcpart_date.`date` = '2008-04-08' and srcpart.hr = 13; + +-- union + subquery +EXPLAIN select distinct(ds) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +PREHOOK: type: QUERY +POSTHOOK: query: -- Disabled until TEZ-1486 is fixed +-- select count(*) from srcpart join srcpart_date on (srcpart.ds = srcpart_date.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) +-- where srcpart_date.`date` = '2008-04-08' and srcpart.hr = 13; + +-- union + subquery +EXPLAIN select distinct(ds) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark + Edges: + Reducer 11 <- Map 10 (GROUP, 1) + Reducer 9 <- Map 8 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 10 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(_col0) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Map 8 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(_col0) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Reducer 11 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + Reducer 9 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 2 Data size: 168 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + + Stage: Stage-1 + Spark + Edges: + Reducer 5 <- Map 4 (GROUP, 1) + Reducer 7 <- Map 6 (GROUP, 1) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2), Reducer 7 (PARTITION-LEVEL SORT, 2) + Reducer 3 <- Reducer 2 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: ds is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 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: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Map 4 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(_col0) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Map 6 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(_col0) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: string) + 1 _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2200 Data size: 23372 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: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE + Reducer 3 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1100 Data size: 11686 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 + Execution mode: vectorized + Reducer 5 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 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: 168 Basic stats: COMPLETE Column stats: NONE + Reducer 7 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 2 Data size: 168 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: 168 Basic stats: COMPLETE Column stats: NONE + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select distinct(ds) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +#### A masked pattern was here #### +POSTHOOK: query: select distinct(ds) from srcpart where srcpart.ds in (select max(srcpart.ds) from srcpart union all select min(srcpart.ds) from srcpart) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 #### +2008-04-08 +2008-04-09 +PREHOOK: query: -- different file format +create table srcpart_orc (key int, value string) partitioned by (ds string, hr int) stored as orc +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@srcpart_orc +POSTHOOK: query: -- different file format +create table srcpart_orc (key int, value string) partitioned by (ds string, hr int) stored as orc +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@srcpart_orc +PREHOOK: query: insert into table srcpart_orc partition (ds, hr) select key, value, ds, hr from srcpart +PREHOOK: type: QUERY +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: Input: default@srcpart@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +PREHOOK: Output: default@srcpart_orc +POSTHOOK: query: insert into table srcpart_orc partition (ds, hr) select key, value, ds, hr from srcpart +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +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 +POSTHOOK: Output: default@srcpart_orc@ds=2008-04-08/hr=11 +POSTHOOK: Output: default@srcpart_orc@ds=2008-04-08/hr=12 +POSTHOOK: Output: default@srcpart_orc@ds=2008-04-09/hr=11 +POSTHOOK: Output: default@srcpart_orc@ds=2008-04-09/hr=12 +POSTHOOK: Lineage: srcpart_orc PARTITION(ds=2008-04-08,hr=11).key EXPRESSION [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: srcpart_orc PARTITION(ds=2008-04-08,hr=11).value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: srcpart_orc PARTITION(ds=2008-04-08,hr=12).key EXPRESSION [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: srcpart_orc PARTITION(ds=2008-04-08,hr=12).value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: srcpart_orc PARTITION(ds=2008-04-09,hr=11).key EXPRESSION [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: srcpart_orc PARTITION(ds=2008-04-09,hr=11).value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: srcpart_orc PARTITION(ds=2008-04-09,hr=12).key EXPRESSION [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: srcpart_orc PARTITION(ds=2008-04-09,hr=12).value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: EXPLAIN select count(*) from srcpart_orc join srcpart_date_hour on (srcpart_orc.ds = srcpart_date_hour.ds and srcpart_orc.hr = srcpart_date_hour.hr) where srcpart_date_hour.hour = 11 and (srcpart_date_hour.`date` = '2008-04-08' or srcpart_date_hour.`date` = '2008-04-09') +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN select count(*) from srcpart_orc join srcpart_date_hour on (srcpart_orc.ds = srcpart_date_hour.ds and srcpart_orc.hr = srcpart_date_hour.hr) where srcpart_date_hour.hour = 11 and (srcpart_date_hour.`date` = '2008-04-08' or srcpart_date_hour.`date` = '2008-04-09') +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 3 + Map Operator Tree: + TableScan + alias: srcpart_date_hour + filterExpr: (((ds is not null and UDFToDouble(hr) is not null) and (hour = 11)) and ((date = '2008-04-08') or (date = '2008-04-09'))) (type: boolean) + Statistics: Num rows: 4 Data size: 1440 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (((ds is not null and UDFToDouble(hr) is not null) and (hour = 11)) and ((date = '2008-04-08') or (date = '2008-04-09'))) (type: boolean) + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 ds (type: string), UDFToDouble(hr) (type: double) + 1 ds (type: string), UDFToDouble(hr) (type: double) + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: ds + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + target column name: ds + target work: Map 1 + Select Operator + expressions: UDFToDouble(hr) (type: double) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: double) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + partition key expr: UDFToDouble(hr) + Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: NONE + target column name: hr + target work: Map 1 + Local Work: + Map Reduce Local Work + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart_orc + filterExpr: UDFToDouble(hr) is not null (type: boolean) + Statistics: Num rows: 2000 Data size: 188000 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: UDFToDouble(hr) is not null (type: boolean) + Statistics: Num rows: 1000 Data size: 94000 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 ds (type: string), UDFToDouble(hr) (type: double) + 1 ds (type: string), UDFToDouble(hr) (type: double) + input vertices: + 1 Map 3 + Statistics: Num rows: 1100 Data size: 103400 Basic stats: COMPLETE 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) + Local Work: + Map Reduce Local Work + 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 + 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 + Processor Tree: + ListSink + +PREHOOK: query: select count(*) from srcpart_orc join srcpart_date_hour on (srcpart_orc.ds = srcpart_date_hour.ds and srcpart_orc.hr = srcpart_date_hour.hr) where srcpart_date_hour.hour = 11 and (srcpart_date_hour.`date` = '2008-04-08' or srcpart_date_hour.`date` = '2008-04-09') +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart_date_hour +PREHOOK: Input: default@srcpart_orc +PREHOOK: Input: default@srcpart_orc@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpart_orc@ds=2008-04-08/hr=12 +PREHOOK: Input: default@srcpart_orc@ds=2008-04-09/hr=11 +PREHOOK: Input: default@srcpart_orc@ds=2008-04-09/hr=12 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart_orc join srcpart_date_hour on (srcpart_orc.ds = srcpart_date_hour.ds and srcpart_orc.hr = srcpart_date_hour.hr) where srcpart_date_hour.hour = 11 and (srcpart_date_hour.`date` = '2008-04-08' or srcpart_date_hour.`date` = '2008-04-09') +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart_date_hour +POSTHOOK: Input: default@srcpart_orc +POSTHOOK: Input: default@srcpart_orc@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpart_orc@ds=2008-04-08/hr=12 +POSTHOOK: Input: default@srcpart_orc@ds=2008-04-09/hr=11 +POSTHOOK: Input: default@srcpart_orc@ds=2008-04-09/hr=12 +#### A masked pattern was here #### +1000 +PREHOOK: query: select count(*) from srcpart where (ds = '2008-04-08' or ds = '2008-04-09') and hr = 11 +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from srcpart where (ds = '2008-04-08' or ds = '2008-04-09') and hr = 11 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 +#### A masked pattern was here #### +1000 +PREHOOK: query: drop table srcpart_orc +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@srcpart_orc +PREHOOK: Output: default@srcpart_orc +POSTHOOK: query: drop table srcpart_orc +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@srcpart_orc +POSTHOOK: Output: default@srcpart_orc +PREHOOK: query: drop table srcpart_date +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@srcpart_date +PREHOOK: Output: default@srcpart_date +POSTHOOK: query: drop table srcpart_date +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@srcpart_date +POSTHOOK: Output: default@srcpart_date +PREHOOK: query: drop table srcpart_hour +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@srcpart_hour +PREHOOK: Output: default@srcpart_hour +POSTHOOK: query: drop table srcpart_hour +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@srcpart_hour +POSTHOOK: Output: default@srcpart_hour +PREHOOK: query: drop table srcpart_date_hour +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@srcpart_date_hour +PREHOOK: Output: default@srcpart_date_hour +POSTHOOK: query: drop table srcpart_date_hour +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@srcpart_date_hour +POSTHOOK: Output: default@srcpart_date_hour +PREHOOK: query: drop table srcpart_double_hour +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@srcpart_double_hour +PREHOOK: Output: default@srcpart_double_hour +POSTHOOK: query: drop table srcpart_double_hour +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@srcpart_double_hour +POSTHOOK: Output: default@srcpart_double_hour diff --git ql/src/test/results/clientpositive/spark/udf_example_add.q.out ql/src/test/results/clientpositive/spark/udf_example_add.q.out index 7916679..587c68b 100644 --- ql/src/test/results/clientpositive/spark/udf_example_add.q.out +++ ql/src/test/results/clientpositive/spark/udf_example_add.q.out @@ -34,14 +34,11 @@ STAGE PLANS: Processor Tree: TableScan alias: src - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: 3 (type: int), 6 (type: int), 10 (type: int), 3.3000000000000003 (type: double), 6.6 (type: double), 11.0 (type: double), 10.4 (type: double) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 - Statistics: Num rows: 500 Data size: 22000 Basic stats: COMPLETE Column stats: COMPLETE Limit Number of rows: 1 - Statistics: Num rows: 1 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE ListSink PREHOOK: query: SELECT example_add(1, 2), diff --git ql/src/test/results/clientpositive/spark/udf_in_file.q.out ql/src/test/results/clientpositive/spark/udf_in_file.q.out index 5ba62b0..be0627c 100644 --- ql/src/test/results/clientpositive/spark/udf_in_file.q.out +++ ql/src/test/results/clientpositive/spark/udf_in_file.q.out @@ -57,14 +57,11 @@ STAGE PLANS: Processor Tree: TableScan alias: value_src - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: in_file(str_val, '../../data/files/test2.dat') (type: boolean), in_file(ch_val, '../../data/files/test2.dat') (type: boolean), in_file(vch_val, '../../data/files/test2.dat') (type: boolean), in_file(str_val_neg, '../../data/files/test2.dat') (type: boolean), in_file(ch_val_neg, '../../data/files/test2.dat') (type: boolean), in_file(vch_val_neg, '../../data/files/test2.dat') (type: boolean), in_file('303', '../../data/files/test2.dat') (type: boolean), in_file('304', '../../data/files/test2.dat') (type: boolean), in_file(null, '../../data/files/test2.dat') (type: boolean) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE Limit Number of rows: 1 - Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE ListSink PREHOOK: query: SELECT in_file(str_val, "../../data/files/test2.dat"), diff --git ql/src/test/results/clientpositive/spark/union_view.q.out ql/src/test/results/clientpositive/spark/union_view.q.out index 31e0786..cce7710 100644 --- ql/src/test/results/clientpositive/spark/union_view.q.out +++ ql/src/test/results/clientpositive/spark/union_view.q.out @@ -54,14 +54,11 @@ STAGE PLANS: TableScan alias: src_union_1 filterExpr: ((key = 86) and (ds = '1')) (type: boolean) - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (key = 86) (type: boolean) - Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: 86 (type: int), value (type: string), '1' (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE ListSink STAGE DEPENDENCIES: @@ -75,14 +72,11 @@ STAGE PLANS: TableScan alias: src_union_2 filterExpr: ((key = 86) and (ds = '2')) (type: boolean) - Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (key = 86) (type: boolean) - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: 86 (type: int), value (type: string), '2' (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE ListSink STAGE DEPENDENCIES: @@ -96,14 +90,11 @@ STAGE PLANS: TableScan alias: src_union_3 filterExpr: ((key = 86) and (ds = '3')) (type: boolean) - Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (key = 86) (type: boolean) - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: 86 (type: int), value (type: string), '3' (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE ListSink 86 val_86 1 diff --git ql/src/test/results/clientpositive/spark/vector_elt.q.out ql/src/test/results/clientpositive/spark/vector_elt.q.out index 49d1458..bb66867 100644 --- ql/src/test/results/clientpositive/spark/vector_elt.q.out +++ ql/src/test/results/clientpositive/spark/vector_elt.q.out @@ -16,17 +16,13 @@ STAGE PLANS: Processor Tree: TableScan alias: alltypesorc - Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (ctinyint > 0) (type: boolean) - Statistics: Num rows: 4096 Data size: 125745 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: ((UDFToInteger(ctinyint) % 2) + 1) (type: int), cstring1 (type: string), cint (type: int), elt(((UDFToInteger(ctinyint) % 2) + 1), cstring1, cint) (type: string) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 4096 Data size: 125745 Basic stats: COMPLETE Column stats: NONE Limit Number of rows: 10 - Statistics: Num rows: 10 Data size: 300 Basic stats: COMPLETE Column stats: NONE ListSink PREHOOK: query: SELECT (ctinyint % 2) + 1, cstring1, cint, elt((ctinyint % 2) + 1, cstring1, cint) @@ -87,14 +83,11 @@ STAGE PLANS: Processor Tree: TableScan alias: alltypesorc - Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: 'defg' (type: string), 'cc' (type: string), 'abc' (type: string), '2' (type: string), '12345' (type: string), '123456789012' (type: string), '1.25' (type: string), '16.0' (type: string), null (type: string), null (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9 - Statistics: Num rows: 12288 Data size: 8687616 Basic stats: COMPLETE Column stats: COMPLETE Limit Number of rows: 1 - Statistics: Num rows: 1 Data size: 707 Basic stats: COMPLETE Column stats: COMPLETE ListSink PREHOOK: query: SELECT elt(2, 'abc', 'defg'), 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 b78aec9..5a2f9e4 100644 --- ql/src/test/results/clientpositive/spark/vector_string_concat.q.out +++ ql/src/test/results/clientpositive/spark/vector_string_concat.q.out @@ -117,14 +117,11 @@ STAGE PLANS: Processor Tree: TableScan alias: over1korc - 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: 1049 Data size: 311170 Basic stats: COMPLETE Column stats: NONE Limit Number of rows: 20 - 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/vectorization_decimal_date.q.out ql/src/test/results/clientpositive/spark/vectorization_decimal_date.q.out index 19799b3..c20033c 100644 --- ql/src/test/results/clientpositive/spark/vectorization_decimal_date.q.out +++ ql/src/test/results/clientpositive/spark/vectorization_decimal_date.q.out @@ -22,17 +22,13 @@ STAGE PLANS: Processor Tree: TableScan alias: date_decimal_test - 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: 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: 3072 Data size: 412815 Basic stats: COMPLETE Column stats: NONE Limit Number of rows: 10 - 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_div0.q.out ql/src/test/results/clientpositive/spark/vectorization_div0.q.out index 5887839..71f4b09 100644 --- ql/src/test/results/clientpositive/spark/vectorization_div0.q.out +++ ql/src/test/results/clientpositive/spark/vectorization_div0.q.out @@ -16,14 +16,11 @@ STAGE PLANS: Processor Tree: TableScan alias: alltypesorc - Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: (cdouble / 0.0) (type: double) outputColumnNames: _col0 - Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE Limit Number of rows: 100 - Statistics: Num rows: 100 Data size: 3000 Basic stats: COMPLETE Column stats: NONE ListSink PREHOOK: query: select cdouble / 0.0 from alltypesorc limit 100 diff --git ql/src/test/results/clientpositive/spark/vectorized_case.q.out ql/src/test/results/clientpositive/spark/vectorized_case.q.out index 0e1e9d3..7f824f1 100644 --- ql/src/test/results/clientpositive/spark/vectorized_case.q.out +++ ql/src/test/results/clientpositive/spark/vectorized_case.q.out @@ -44,14 +44,11 @@ STAGE PLANS: Processor Tree: TableScan alias: alltypesorc - Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: ((csmallint = 418) or ((csmallint = 12205) or (csmallint = 10583))) (type: boolean) - Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: csmallint (type: smallint), CASE WHEN ((csmallint = 418)) THEN ('a') WHEN ((csmallint = 12205)) THEN ('b') ELSE ('c') END (type: string), CASE (csmallint) WHEN (418) THEN ('a') WHEN (12205) THEN ('b') ELSE ('c') END (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE ListSink PREHOOK: query: select diff --git ql/src/test/results/clientpositive/spark/vectorized_math_funcs.q.out ql/src/test/results/clientpositive/spark/vectorized_math_funcs.q.out index 470d9a9..444b9e0 100644 --- ql/src/test/results/clientpositive/spark/vectorized_math_funcs.q.out +++ ql/src/test/results/clientpositive/spark/vectorized_math_funcs.q.out @@ -116,14 +116,11 @@ STAGE PLANS: Processor Tree: TableScan alias: alltypesorc - Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (((cbigint % 500) = 0) and (sin(cfloat) >= -1.0)) (type: boolean) - Statistics: Num rows: 2048 Data size: 62872 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: cdouble (type: double), round(cdouble, 2) (type: double), floor(cdouble) (type: bigint), ceil(cdouble) (type: bigint), rand() (type: double), rand(98007) (type: double), exp(ln(cdouble)) (type: double), ln(cdouble) (type: double), ln(cfloat) (type: double), log10(cdouble) (type: double), log2(cdouble) (type: double), log2((cdouble - 15601.0)) (type: double), log2(cfloat) (type: double), log2(cbigint) (type: double), log2(cint) (type: double), log2(csmallint) (type: double), log2(ctinyint) (type: double), log(2.0, cdouble) (type: double), power(log2(cdouble), 2.0) (type: double), power(log2(cdouble), 2.0) (type: double), sqrt(cdouble) (type: double), sqrt(cbigint) (type: double), bin(cbigint) (type: string), hex(cdouble) (type: string), conv(cbigint, 10, 16) (type: string), abs(cdouble) (type: double), abs(ctinyint) (type: int), (cint pmod 3) (type: int), sin(cdouble) (type: double), asin(cdouble) (type: double), cos(cdouble) (type: double), acos(cdouble) (type: double), atan(cdouble) (type: double), degrees(cdouble) (type: double), radians(cdouble) (type: double), cdouble (type: double), cbigint (type: bigint), (- cdouble) (type: double), sign(cdouble) (type: double), sign(cbigint) (type: double), cos(((- sin(log(cdouble))) + 3.14159)) (type: double) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27, _col28, _col29, _col30, _col31, _col32, _col33, _col34, _col35, _col36, _col37, _col38, _col39, _col40 - Statistics: Num rows: 2048 Data size: 62872 Basic stats: COMPLETE Column stats: NONE ListSink PREHOOK: query: select diff --git ql/src/test/results/clientpositive/spark/vectorized_string_funcs.q.out ql/src/test/results/clientpositive/spark/vectorized_string_funcs.q.out index ef98ae9..0463d31 100644 --- ql/src/test/results/clientpositive/spark/vectorized_string_funcs.q.out +++ ql/src/test/results/clientpositive/spark/vectorized_string_funcs.q.out @@ -56,14 +56,11 @@ STAGE PLANS: Processor Tree: TableScan alias: alltypesorc - Statistics: Num rows: 12288 Data size: 377237 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: ((((cbigint % 237) = 0) and (length(substr(cstring1, 1, 2)) <= 2)) and (cstring1 like '%')) (type: boolean) - Statistics: Num rows: 1024 Data size: 31436 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: substr(cstring1, 1, 2) (type: string), substr(cstring1, 2) (type: string), lower(cstring1) (type: string), upper(cstring1) (type: string), upper(cstring1) (type: string), length(cstring1) (type: int), trim(cstring1) (type: string), ltrim(cstring1) (type: string), rtrim(cstring1) (type: string), concat(cstring1, cstring2) (type: string), concat('>', cstring1) (type: string), concat(cstring1, '<') (type: string), concat(substr(cstring1, 1, 2), substr(cstring2, 1, 2)) (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12 - Statistics: Num rows: 1024 Data size: 31436 Basic stats: COMPLETE Column stats: NONE ListSink PREHOOK: query: select