diff --git common/src/java/org/apache/hadoop/hive/conf/HiveConf.java common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index e138800..e3fe853 100644 --- common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -2059,6 +2059,10 @@ 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."), 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 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..8246968 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 @@ -6,11 +6,6 @@ */ package org.apache.hadoop.hive.ql.plan.api; - -import java.util.Map; -import java.util.HashMap; -import org.apache.thrift.TEnum; - public enum OperatorType implements org.apache.thrift.TEnum { JOIN(0), MAPJOIN(1), @@ -36,7 +31,8 @@ EVENT(21), ORCFILEMERGE(22), RCFILEMERGE(23), - MERGEJOIN(24); + MERGEJOIN(24), + SPPS(25); private final int value; @@ -107,6 +103,8 @@ public static OperatorType findByValue(int value) { return RCFILEMERGE; case 24: return MERGEJOIN; + case 25: + return SPPS; 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 91e8a02..223d9b0 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java @@ -35,6 +35,8 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorSelectOperator; 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; @@ -116,6 +118,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, 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..fcdc3c2 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkDynamicPartitionPruner.java @@ -0,0 +1,261 @@ +/** + * 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.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); + } + + private void processFiles(MapWork work, JobConf jobConf) throws HiveException { + try { + Path baseDir = work.getTmpPathForPartitionPruning(); + FileSystem fs = FileSystem.get(baseDir.toUri(), jobConf); + ObjectInputStream in = null; + + // 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); + } + } + } + in.close(); + } catch (Exception e) { + throw new HiveException(e); + } + } + + 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 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(); + LOG.debug("Type of obj insp: " + inspector.getTypeName()); + + soi = (StructObjectInspector) inspector; + List fields = soi.getAllStructFieldRefs(); + if (fields.size() > 1) { + LOG.error("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 72ab913..0e11da3 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 @@ -21,15 +21,20 @@ import java.io.IOException; import java.net.URI; import java.net.URISyntaxException; +import java.util.Collection; import org.apache.commons.io.FilenameUtils; 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.SparkConf; @@ -113,4 +118,46 @@ public static SparkSession getSparkSession(HiveConf conf, SessionState.get().setSparkSession(sparkSession); return sparkSession; } + + /** + * Generate a temporary path for dynamic partition pruning in Spark branch + * TODO: no longer need this if we use broadcast variable or 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); + } + + public static void collectOp(Collection> coll, Operator root, Class clazz) { + if (root == null) { + return; + } + if (clazz.equals(root.getClass())) { + coll.add(root); + } + for (Operator child : root.getChildOperators()) { + collectOp(coll, child, clazz); + } + } } 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..6cf7d47 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; @@ -506,7 +507,17 @@ public int hashCode() { ArrayList result = new ArrayList(); - Path[] paths = getInputPaths(job); + Path[] paths; + if (HiveConf.getVar(job, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE).equals("spark")) { + Set pathStrings = mrwork.getPathToPartitionInfo().keySet(); + paths = new Path[pathStrings.size()]; + Iterator it = pathStrings.iterator(); + for (int i = 0; i < paths.length; i++) { + paths[i] = new Path(it.next()); + } + } else { + paths = getInputPaths(job); + } List nonCombinablePaths = new ArrayList(paths.length / 2); List combinablePaths = new ArrayList(paths.length / 2); @@ -514,36 +525,38 @@ 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]); + + 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 String oldPaths = job.get(HiveConf.ConfVars.HADOOPMAPREDINPUTDIR.varname); if (LOG.isDebugEnabled()) { 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 9d5730d..a9efab1 100755 --- ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java +++ ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java @@ -27,6 +27,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; @@ -38,6 +39,8 @@ 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.metadata.HiveException; import org.apache.hadoop.hive.ql.plan.TableDesc; import org.apache.hadoop.hive.ql.exec.Operator; import org.apache.hadoop.hive.ql.exec.TableScanOperator; @@ -267,6 +270,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(); } } diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/SparkDynamicPartitionPruningOptimization.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/SparkDynamicPartitionPruningOptimization.java new file mode 100644 index 0000000..8db93bd --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/SparkDynamicPartitionPruningOptimization.java @@ -0,0 +1,343 @@ +/** + * 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.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Stack; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.exec.FilterOperator; +import org.apache.hadoop.hive.ql.exec.GroupByOperator; +import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.hive.ql.exec.OperatorFactory; +import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; +import org.apache.hadoop.hive.ql.exec.SelectOperator; +import org.apache.hadoop.hive.ql.exec.TableScanOperator; +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.GraphWalker; +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.lib.Rule; +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.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.ExprNodeColumnDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeDynamicListDesc; +import org.apache.hadoop.hive.ql.plan.FilterDesc; +import org.apache.hadoop.hive.ql.plan.GroupByDesc; +import org.apache.hadoop.hive.ql.plan.OperatorDesc; +import org.apache.hadoop.hive.ql.plan.PlanUtils; +import org.apache.hadoop.hive.ql.plan.SelectDesc; + +/** + * This optimization looks for expressions of the kind "x IN (RS[n])". If such + * an expression made it to a table scan operator and x is a partition column we + * can use an existing join to dynamically prune partitions. This class sets up + * the infrastructure for that. + * + * Copied from DynamicPartitionPruningOptimization + */ +public class SparkDynamicPartitionPruningOptimization implements NodeProcessor { + + static final private Log LOG = LogFactory.getLog(SparkDynamicPartitionPruningOptimization.class + .getName()); + + public static class DynamicPartitionPrunerProc implements NodeProcessor { + + /** + * process simply remembers all the dynamic partition pruning expressions + * found + */ + @Override + public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, + Object... nodeOutputs) throws SemanticException { + ExprNodeDynamicListDesc desc = (ExprNodeDynamicListDesc) nd; + DynamicPartitionPrunerContext context = (DynamicPartitionPrunerContext) procCtx; + + // Rule is searching for dynamic pruning expr. There's at least an IN + // expression wrapping it. + ExprNodeDesc parent = (ExprNodeDesc) stack.get(stack.size() - 2); + ExprNodeDesc grandParent = stack.size() >= 3 ? + (ExprNodeDesc) stack.get(stack.size() - 3) : null; + + context.addDynamicList(desc, parent, grandParent, (ReduceSinkOperator) desc.getSource()); + + return context; + } + } + + private static class DynamicListContext { + public ExprNodeDynamicListDesc desc; + public ExprNodeDesc parent; + public ExprNodeDesc grandParent; + public ReduceSinkOperator generator; + + public DynamicListContext(ExprNodeDynamicListDesc desc, ExprNodeDesc parent, + ExprNodeDesc grandParent, ReduceSinkOperator generator) { + this.desc = desc; + this.parent = parent; + this.grandParent = grandParent; + this.generator = generator; + } + } + + private static class DynamicPartitionPrunerContext implements NodeProcessorCtx, + Iterable { + public List dynLists = new ArrayList(); + + public void addDynamicList(ExprNodeDynamicListDesc desc, ExprNodeDesc parent, + ExprNodeDesc grandParent, ReduceSinkOperator generator) { + dynLists.add(new DynamicListContext(desc, parent, grandParent, generator)); + } + + @Override + public Iterator iterator() { + return dynLists.iterator(); + } + } + + private String extractColName(ExprNodeDesc root) { + if (root instanceof ExprNodeColumnDesc) { + return ((ExprNodeColumnDesc) root).getColumn(); + } else { + if (root.getChildren() == null) { + return null; + } + + String column = null; + for (ExprNodeDesc d: root.getChildren()) { + String candidate = extractColName(d); + if (column != null && candidate != null) { + return null; + } else if (candidate != null) { + column = candidate; + } + } + return column; + } + } + + @Override + public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, Object... no) + throws SemanticException { + OptimizeSparkProcContext context = (OptimizeSparkProcContext) procCtx; + ParseContext parseContext = context.getParseContext(); + + FilterOperator filter = (FilterOperator) nd; + FilterDesc desc = filter.getConf(); + TableScanOperator ts = null; + + DynamicPartitionPrunerContext removerContext = new DynamicPartitionPrunerContext(); + + if (filter.getParentOperators().size() == 1 + && filter.getParentOperators().get(0) instanceof TableScanOperator) { + ts = (TableScanOperator) filter.getParentOperators().get(0); + } + + if (LOG.isDebugEnabled()) { + LOG.debug("Parent: " + filter.getParentOperators().get(0)); + LOG.debug("Filter: " + desc.getPredicateString()); + LOG.debug("TableScan: " + ts); + } + + // collect the dynamic pruning conditions + removerContext.dynLists.clear(); + walkExprTree(desc.getPredicate(), removerContext); + + for (DynamicListContext ctx : removerContext) { + String column = extractColName(ctx.parent); + + if (ts != null && column != null) { + Table table = ts.getConf().getTableMetadata(); + + if (table != null && table.isPartitionKey(column)) { + String alias = ts.getConf().getAlias(); + PrunedPartitionList plist = parseContext.getPrunedPartitions(alias, ts); + if (LOG.isDebugEnabled()) { + LOG.debug("alias: " + alias); + LOG.debug("pruned partition list: "); + if (plist != null) { + for (Partition p : plist.getPartitions()) { + LOG.debug(p.getCompleteName()); + } + } + } + if (plist == null || plist.getPartitions().size() != 0) { + LOG.info("Dynamic partitioning: " + table.getCompleteName() + "." + column); + generatePartitionPrunerPlan(ctx, parseContext, ts, column); + } else { + // all partitions have been statically removed + LOG.debug("No partition pruning necessary."); + } + } else { + LOG.debug("Column " + column + " is not a partition column"); + } + } + + // we always remove the condition by replacing it with "true" + ExprNodeDesc constNode = new ExprNodeConstantDesc(ctx.parent.getTypeInfo(), true); + if (ctx.grandParent == null) { + desc.setPredicate(constNode); + } else { + int i = ctx.grandParent.getChildren().indexOf(ctx.parent); + ctx.grandParent.getChildren().remove(i); + ctx.grandParent.getChildren().add(i, constNode); + } + } + + // if we pushed the predicate into the table scan we need to remove the + // synthetic conditions there. + cleanTableScanFilters(ts); + + return false; + } + + private void cleanTableScanFilters(TableScanOperator ts) throws SemanticException { + + if (ts == null || ts.getConf() == null || ts.getConf().getFilterExpr() == null) { + // nothing to do + return; + } + + DynamicPartitionPrunerContext removerContext = new DynamicPartitionPrunerContext(); + + // collect the dynamic pruning conditions + removerContext.dynLists.clear(); + walkExprTree(ts.getConf().getFilterExpr(), removerContext); + + for (DynamicListContext ctx : removerContext) { + // remove the condition by replacing it with "true" + ExprNodeDesc constNode = new ExprNodeConstantDesc(ctx.parent.getTypeInfo(), true); + if (ctx.grandParent == null) { + // we're the only node, just clear out the expression + ts.getConf().setFilterExpr(null); + } else { + int i = ctx.grandParent.getChildren().indexOf(ctx.parent); + ctx.grandParent.getChildren().remove(i); + ctx.grandParent.getChildren().add(i, constNode); + } + } + } + + private void generatePartitionPrunerPlan(DynamicListContext ctx, ParseContext parseContext, + TableScanOperator ts, String column) { + + // we will put a fork in the plan at the source of the reduce sink + Operator parentOfRS = ctx.generator.getParentOperators().get(0); + + // we need the expr that generated the key of the reduce sink + ExprNodeDesc key = ctx.generator.getConf().getKeyCols().get(ctx.desc.getKeyIndex()); + + // we also need the expr for the partitioned table + ExprNodeDesc partKey = ctx.parent.getChildren().get(0); + + if (LOG.isDebugEnabled()) { + LOG.debug("key expr: " + key); + LOG.debug("partition key expr: " + partKey); + } + + List keyExprs = new ArrayList(); + keyExprs.add(key); + + // group by requires "ArrayList", don't ask. + ArrayList outputNames = new ArrayList(); + outputNames.add(HiveConf.getColumnInternalName(0)); + + // project the relevant key column + SelectDesc select = new SelectDesc(keyExprs, outputNames); + SelectOperator selectOp = + (SelectOperator) OperatorFactory.getAndMakeChild(select, parentOfRS); + + // do a group by on the list to dedup + float groupByMemoryUsage = + HiveConf.getFloatVar(parseContext.getConf(), HiveConf.ConfVars.HIVEMAPAGGRHASHMEMORY); + float memoryThreshold = + HiveConf.getFloatVar(parseContext.getConf(), + HiveConf.ConfVars.HIVEMAPAGGRMEMORYTHRESHOLD); + + ArrayList groupByExprs = new ArrayList(); + ExprNodeDesc groupByExpr = + new ExprNodeColumnDesc(key.getTypeInfo(), outputNames.get(0), null, false); + groupByExprs.add(groupByExpr); + + GroupByDesc groupBy = + new GroupByDesc(GroupByDesc.Mode.HASH, outputNames, groupByExprs, + new ArrayList(), false, groupByMemoryUsage, memoryThreshold, + null, false, 0, true); + + GroupByOperator groupByOp = + (GroupByOperator) OperatorFactory.getAndMakeChild(groupBy, selectOp); + + Map colMap = new HashMap(); + colMap.put(outputNames.get(0), groupByExpr); + groupByOp.setColumnExprMap(colMap); + + // finally add partition pruner sink operator + 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) + 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 exprRules = new LinkedHashMap(); + exprRules.put(new RuleRegExp("R1", ExprNodeDynamicListDesc.class.getName() + "%"), + new DynamicPartitionPrunerProc()); + + // The dispatcher fires the processor corresponding to the closest matching + // rule and passes the context along + Dispatcher disp = new DefaultRuleDispatcher(null, exprRules, ctx); + GraphWalker egw = new DefaultGraphWalker(disp); + + List startNodes = new ArrayList(); + startNodes.add(pred); + + HashMap outputMap = new HashMap(); + egw.startWalking(startNodes, outputMap); + return outputMap; + } + +} diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkMapJoinResolver.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkMapJoinResolver.java index 8e56263..de251db 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkMapJoinResolver.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkMapJoinResolver.java @@ -131,11 +131,15 @@ private boolean containsOp(SparkWork sparkWork, Class clazz) { // of its parent SparkWorks for the small tables private final Map> dependencyGraph; + // To avoid newly generated SparkTask to be processed. + private final Set generatedTaskSet; + public SparkMapJoinTaskDispatcher(PhysicalContext pc) { super(); physicalContext = pc; sparkWorkMap = new LinkedHashMap(); dependencyGraph = new LinkedHashMap>(); + generatedTaskSet = new HashSet(); } // Move the specified work from the sparkWork to the targetWork @@ -292,6 +296,7 @@ private SparkTask createSparkTask(SparkTask originalTask, } createdTaskMap.put(sparkWork, resultTask); + generatedTaskSet.add(resultTask); return resultTask; } @@ -299,7 +304,7 @@ private SparkTask createSparkTask(SparkTask originalTask, public Object dispatch(Node nd, Stack stack, Object... nos) throws SemanticException { Task currentTask = (Task) nd; - if(currentTask.isMapRedTask()) { + if(currentTask.isMapRedTask() && !generatedTaskSet.contains(currentTask)) { if (currentTask instanceof ConditionalTask) { List> taskList = ((ConditionalTask) currentTask).getListTasks(); 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..cb62d10 --- /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") + 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..f560f2b 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,12 @@ // 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 +159,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 +182,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 e27ce0d..352bd8c 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 @@ -218,6 +221,13 @@ 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 + if (context.rootToWorkMap.containsKey(op)) { + context.rootToWorkMap.put(newOp, context.rootToWorkMap.get(op)); + } + // Don't remove the old entry + if (op instanceof FileSinkOperator) { List fileSinkList = context.fileSinkMap.get(op); if (fileSinkList == null) { @@ -225,6 +235,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 +353,68 @@ 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); + + if (targetWork == null) { + throw new AssertionError("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 +568,35 @@ 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..64cdd28 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 @@ -49,19 +49,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 +77,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 19aae70..3889afe 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 @@ -35,6 +35,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 +45,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 +61,8 @@ 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.SparkDynamicPartitionPruningOptimization; 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; @@ -82,7 +86,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. @@ -101,20 +104,28 @@ 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 + OptimizeSparkProcContext procCtx = new OptimizeSparkProcContext(conf, pCtx, inputs, outputs); + + // Run Spark Dynamic Partition Pruning + runDynamicPartitionPruning(procCtx); + + // Run Join related optimizations + runJoinOptimizations(procCtx); + + PERF_LOGGER.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_OPTIMIZE_OPERATOR_TREE); + } + + private void runJoinOptimizations(OptimizeSparkProcContext procCtx) throws SemanticException { + ParseContext pCtx = procCtx.getParseContext(); + + // Create a walker which walks the tree in a DFS manner while maintaining // the operator stack. 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)); // The dispatcher fires the processor corresponding to the closest matching @@ -126,7 +137,34 @@ 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); + } + + 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 SparkDynamicPartitionPruningOptimization()); + + // 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(parseContext.getTopOps().values()); + GraphWalker ogw = new ForwardWalker(disp); + 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); + } } /** @@ -137,28 +175,93 @@ 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 operator tree if necessary. + + Map opRules = new LinkedHashMap(); + opRules.put(new RuleRegExp("Clone OP tree for PartitionPruningSink", + SparkPartitionPruningSinkOperator.getOperatorName() + "%"), + new SparkPartitionPruningOptimizer()); + + 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); + + + // ---------------------- Second Pass -------------------------- // + // Break operator tree, and generate work trees. + + topNodes.clear(); + topNodes.addAll(procCtx.topOps.values()); + generateWorkTree(procCtx, topNodes); + + 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); + generateWorkTree(procCtx, topNodes); + + procCtx.currentTask = mainTask; + } + + // --------------------- Post Process ------------------------- // + + // We need to clone some operator plans and remove union operators still + for (BaseWork w: procCtx.workWithUnionOperators) { + utils.removeUnionOperators(conf, procCtx, w); + } + + // Finally make sure the file sink operators are set up right + for (FileSinkOperator fileSink: procCtx.fileSinkSet) { + utils.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 generateWorkTree(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", - FileSinkOperator.getOperatorName() + "%"), + FileSinkOperator.getOperatorName() + "%"), new CompositeProcessor(new SparkFileSinkProcessor(), genSparkWork)); opRules.put(new RuleRegExp("Handle Analyze Command", - TableScanOperator.getOperatorName() + "%"), + TableScanOperator.getOperatorName() + "%"), new SparkProcessAnalyzeTable(GenSparkUtils.getUtils())); opRules.put(new RuleRegExp("Remember union", UnionOperator.getOperatorName() + "%"), @@ -185,8 +288,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() { @@ -218,25 +323,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 @@ -338,6 +426,5 @@ protected void optimizeTaskPlan(List> rootTasks, Pa } PERF_LOGGER.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_OPTIMIZE_TASK_TREE); - return; } } diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkPartitionPruningOptimizer.java ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkPartitionPruningOptimizer.java new file mode 100644 index 0000000..73c6d22 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkPartitionPruningOptimizer.java @@ -0,0 +1,145 @@ +/** + * 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.HashMap; +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.metadata.Table; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.plan.TableScanDesc; + +/** + * This processor triggers on SparkPartitionPruningSinkOperator. For a operator tree like + * this: + * + * Original Tree: + * TS TS + * | | + * FIL FIL + * | | \ + * RS RS SEL + * \ / | + * JOIN GBY + * | + * SPPS + * + * It removes the branch containing SPPS from the original operator tree, and split it into + * two separate trees: + * + * Tree #1: Tree #2: + * TS TS TS + * | | | + * FIL FIL FIL + * | | | + * RS RS SEL + * \ / | + * JOIN GBY + * | + * SPPS + * + * For MapJoinOperator, this optimizer will not do anything - it should be executed within + * the same SparkTask. + */ +public class SparkPartitionPruningOptimizer 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 + Operator opToBreak = pruningSinkOp, selOp = null; + while (opToBreak != null) { + if (opToBreak.getNumChild() > 1) { + break; + } else { + selOp = opToBreak; + opToBreak = opToBreak.getParentOperators().get(0); + } + } + + // Check if this is a MapJoin. If so, do not split. + for (Operator childOp : opToBreak.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 = opToBreak.getChildOperators(); + opToBreak.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 + opToBreak.setChildOperators(savedChildOps); + opToBreak.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: should only contain one SparkPartitionPruningSinkOperator"); + SparkPartitionPruningSinkOperator clonedPruningSinkOp = + (SparkPartitionPruningSinkOperator) sinkSet.iterator().next(); + clonedPruningSinkOp.getConf().setTableScan(pruningSinkOp.getConf().getTableScan()); + context.pruningSinkSet.add(clonedPruningSinkOp); + + return null; + } + + private void collectRoots(List> ops, Operator op) { + if (op.getNumParent() == 0) { + ops.add(op); + } else { + for (Operator parentOp : op.getParentOperators()) { + collectRoots(ops, parentOp); + } + } + } + +} 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..c837e24 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkPartitionPruningSinkOperator.java @@ -0,0 +1,118 @@ +/** + * 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.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.ql.exec.Operator; +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 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); + } + forward(row, rowInspector); + } + + 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 + + Path path = new Path(conf.getPath(), conf.getTargetColumnName()); + FileSystem fs = path.getFileSystem(this.getConfiguration()); + short numOfRepl = fs.getDefaultReplication(path); + ObjectOutputStream out = null; + + try { + out = new ObjectOutputStream(new BufferedOutputStream(fs.create(path, numOfRepl), 4096)); + out.writeUTF(conf.getTargetColumnName()); + buffer.writeTo(out); + } finally { + if (out != null) { + LOG.info("Flushed to file: " + path); + out.close(); + } + } + } + + @Override + public OperatorType getType() { + return OperatorType.SPPS; + } + + public String getName() { + return getOperatorName(); + } + + public static String getOperatorName() { + return "SPPS"; + } + +} 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 05a5841..d176ace 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; @@ -585,4 +587,12 @@ public void setBaseSrc(String[] baseSrc) { public void setMapAliases(List mapAliases) { this.mapAliases = mapAliases; } + + public Path getTmpPathForPartitionPruning() { + return this.tmpPathForPartitionPruning; + } + + public void setTmpPathForPartitionPruning(Path tmpPathForPartitionPruning) { + this.tmpPathForPartitionPruning = tmpPathForPartitionPruning; + } } 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 aa291b9..f08d4f5 100644 --- ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java +++ ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java @@ -99,7 +99,7 @@ private transient TableSample tableSample; - private transient final Table tableMetadata; + private transient Table tableMetadata; public TableScanDesc() { @@ -281,4 +281,8 @@ public TableSample getTableSample() { public void setTableSample(TableSample tableSample) { this.tableSample = tableSample; } + + public void setTableMetadata(Table tableMetadata) { + this.tableMetadata = tableMetadata; + } } 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..607ba0a 100644 --- ql/src/java/org/apache/hadoop/hive/ql/ppd/SyntheticJoinPredicate.java +++ ql/src/java/org/apache/hadoop/hive/ql/ppd/SyntheticJoinPredicate.java @@ -28,6 +28,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.ql.exec.CommonJoinOperator; import org.apache.hadoop.hive.ql.exec.FunctionRegistry; @@ -67,8 +68,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..7ae84a9 --- /dev/null +++ ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning.q @@ -0,0 +1,178 @@ +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; + + +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/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..601a17f --- /dev/null +++ ql/src/test/results/clientpositive/spark/spark_dynamic_partition_pruning.q.out @@ -0,0 +1,5548 @@ +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: ds + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: ds (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) + Filter Operator + predicate: (ds is not null and (date = '2008-04-08')) (type: boolean) + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Spark Partition Pruning Sink Operator + partition key expr: ds + 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: 1000 Data size: 10624 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: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE 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 + 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 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + 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: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE 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 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + 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) + Filter Operator + predicate: (ds is not null and (date = '2008-04-08')) (type: boolean) + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Spark Partition Pruning Sink Operator + partition key expr: ds + 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) + Filter Operator + predicate: (hr is not null and (hour = 11)) (type: boolean) + Select Operator + expressions: hr (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Spark Partition Pruning Sink Operator + partition key expr: hr + 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: 500 Data size: 5312 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: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE 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: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: hr (type: string) + sort order: + + Map-reduce partition columns: hr (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE 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: 550 Data size: 5843 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: 550 Data size: 5843 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: 605 Data size: 6427 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 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + 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: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE 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: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: hr (type: string) + sort order: + + Map-reduce partition columns: hr (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE 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 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + 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) + Filter Operator + predicate: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Spark Partition Pruning Sink Operator + partition key expr: ds + 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) + Filter Operator + predicate: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Select Operator + expressions: hr (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Spark Partition Pruning Sink Operator + partition key expr: hr + 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: 500 Data size: 5312 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: 0 Data size: 0 Basic stats: NONE 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: 0 Data size: 0 Basic stats: NONE 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: 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 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + 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: 0 Data size: 0 Basic stats: NONE 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: 0 Data size: 0 Basic stats: NONE 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 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + 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) + Filter Operator + predicate: (ds is not null and (date = 'I DONT EXIST')) (type: boolean) + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Spark Partition Pruning Sink Operator + partition key expr: ds + 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: 1000 Data size: 10624 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: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE 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 + 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 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + 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: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE 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 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + 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) + Filter Operator + predicate: (UDFToDouble(UDFToInteger((hr / 2))) is not null and (hour = 11)) (type: boolean) + Select Operator + expressions: UDFToDouble(UDFToInteger((hr / 2))) (type: double) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: double) + mode: hash + outputColumnNames: _col0 + Spark Partition Pruning Sink Operator + partition key expr: UDFToDouble(hr) + 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: 500 Data size: 5312 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: 500 Data size: 5312 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: 0 Data size: 0 Basic stats: NONE 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: 0 Data size: 0 Basic stats: NONE 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: 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 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + 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) + Filter Operator + predicate: (hr is not null and (hour = 11)) (type: boolean) + Select Operator + expressions: hr (type: double) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: double) + mode: hash + outputColumnNames: _col0 + Spark Partition Pruning Sink Operator + partition key expr: (hr * 2) + 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: 500 Data size: 5312 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: 500 Data size: 5312 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: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: hr (type: double) + sort order: + + Map-reduce partition columns: hr (type: double) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE 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: 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 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + 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: 0 Data size: 0 Basic stats: NONE 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: 0 Data size: 0 Basic stats: NONE 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 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + 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: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: hr (type: double) + sort order: + + Map-reduce partition columns: hr (type: double) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE 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 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + 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) + Filter Operator + predicate: (UDFToString(hr) is not null and (hour = 11)) (type: boolean) + Select Operator + expressions: UDFToString(hr) (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Spark Partition Pruning Sink Operator + partition key expr: UDFToString((hr * 2)) + 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: 500 Data size: 5312 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: 500 Data size: 5312 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: 0 Data size: 0 Basic stats: NONE 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: 0 Data size: 0 Basic stats: NONE 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: 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 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + 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 +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 7 <- Map 6 (GROUP, 2) +#### A masked pattern was here #### + Vertices: + Map 6 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: (ds = '2008-04-08') (type: boolean) + Select Operator + expressions: '2008-04-08' (type: string) + outputColumnNames: ds + Group By Operator + keys: ds (type: string) + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Map 8 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: ds is not null (type: boolean) + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Spark Partition Pruning Sink Operator + partition key expr: ds + target column name: ds + target work: Map 1 + Reducer 7 + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Spark Partition Pruning Sink Operator + partition key expr: ds + target column name: ds + target work: Map 5 + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 3 <- Map 5 (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 = '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: ds + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: ds (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 500 Data size: 5312 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: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Map 5 + 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: 1000 Data size: 10624 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: 250 Data size: 2656 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: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 ds (type: string) + 1 _col0 (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 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 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +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 +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 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 +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 #### +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 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + 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) + Filter Operator + predicate: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Spark Partition Pruning Sink Operator + partition key expr: ds + 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) + Filter Operator + predicate: (((ds is not null and hr is not null) and (date = '2008-04-08')) and (hour = 11)) (type: boolean) + Select Operator + expressions: hr (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Spark Partition Pruning Sink Operator + partition key expr: hr + 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: 500 Data size: 5312 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: 0 Data size: 0 Basic stats: NONE 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: 0 Data size: 0 Basic stats: NONE 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: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((_col2 = _col7) and (_col3 = _col9)) (type: boolean) + Statistics: Num rows: 137 Data size: 1455 Basic stats: COMPLETE Column stats: NONE + Select Operator + Statistics: Num rows: 137 Data size: 1455 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 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + 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: 1 Data size: 21 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 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + 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) + Filter Operator + predicate: (date = '2008-04-08') (type: boolean) + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Spark Partition Pruning Sink Operator + partition key expr: ds + 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: 1000 Data size: 10624 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: 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 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + 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 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + 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) + Filter Operator + predicate: (ds is not null and (date = '2008-04-08')) (type: boolean) + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Spark Partition Pruning Sink Operator + partition key expr: ds + 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) + Filter Operator + predicate: ((hr is not null and (hour = 11)) and (hr = 11)) (type: boolean) + Select Operator + expressions: '11' (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Spark Partition Pruning Sink Operator + partition key expr: hr + 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: 250 Data size: 2656 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: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE 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: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: '11' (type: string) + sort order: + + Map-reduce partition columns: '11' (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE 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: 275 Data size: 2921 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: 275 Data size: 2921 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: 302 Data size: 3213 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 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + 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 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: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: ds (type: string) + sort order: + + Map-reduce partition columns: ds (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE 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: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: '13' (type: string) + sort order: + + Map-reduce partition columns: '13' (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE 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: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: '13' (type: string) + sort order: + + Map-reduce partition columns: '13' (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE 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: 0 Data size: 0 Basic stats: NONE 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 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + 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 + Select Operator + expressions: ds (type: string) + outputColumnNames: ds + Group By Operator + aggregations: min(ds) + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: string) + Map 8 + Map Operator Tree: + TableScan + alias: srcpart + Select Operator + expressions: ds (type: string) + outputColumnNames: ds + Group By Operator + aggregations: max(ds) + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: string) + Reducer 11 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Filter Operator + predicate: _col0 is not null (type: boolean) + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Spark Partition Pruning Sink Operator + partition key expr: ds + target column name: ds + target work: Map 1 + Reducer 9 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Filter Operator + predicate: _col0 is not null (type: boolean) + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Spark Partition Pruning Sink Operator + partition key expr: ds + 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 + 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 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: ds + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(ds) + 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: ds + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(ds) + 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 ds (type: string) + 1 _col0 (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 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 5 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Filter Operator + predicate: _col0 is not null (type: boolean) + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Reducer 7 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Filter Operator + predicate: _col0 is not null (type: boolean) + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + + 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 + Select Operator + expressions: ds (type: string) + outputColumnNames: ds + Group By Operator + aggregations: min(ds) + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: string) + Map 8 + Map Operator Tree: + TableScan + alias: srcpart + Select Operator + expressions: ds (type: string) + outputColumnNames: ds + Group By Operator + aggregations: max(ds) + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: string) + Reducer 11 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Filter Operator + predicate: _col0 is not null (type: boolean) + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Spark Partition Pruning Sink Operator + partition key expr: ds + target column name: ds + target work: Map 1 + Reducer 9 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Filter Operator + predicate: _col0 is not null (type: boolean) + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Spark Partition Pruning Sink Operator + partition key expr: ds + 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 + 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 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: ds + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(ds) + 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: ds + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(ds) + 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 ds (type: string) + 1 _col0 (type: string) + outputColumnNames: _col2 + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col2 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1100 Data size: 11686 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: 1100 Data size: 11686 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: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 5 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Filter Operator + predicate: _col0 is not null (type: boolean) + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Reducer 7 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Filter Operator + predicate: _col0 is not null (type: boolean) + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + + 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 14 (GROUP, 1) + Reducer 17 <- Map 16 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 10 + Map Operator Tree: + TableScan + alias: srcpart + Select Operator + expressions: ds (type: string) + outputColumnNames: ds + Group By Operator + aggregations: max(ds) + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: string) + Map 12 + Map Operator Tree: + TableScan + alias: srcpart + Select Operator + expressions: ds (type: string) + outputColumnNames: ds + Group By Operator + aggregations: min(ds) + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: string) + Map 14 + Map Operator Tree: + TableScan + alias: srcpart + Select Operator + expressions: ds (type: string) + outputColumnNames: ds + Group By Operator + aggregations: max(ds) + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: string) + Map 16 + Map Operator Tree: + TableScan + alias: srcpart + Select Operator + expressions: ds (type: string) + outputColumnNames: ds + Group By Operator + aggregations: min(ds) + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: string) + Reducer 11 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Filter Operator + predicate: _col0 is not null (type: boolean) + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Spark Partition Pruning Sink Operator + partition key expr: ds + target column name: ds + target work: Map 1 + Reducer 13 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Filter Operator + predicate: _col0 is not null (type: boolean) + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Spark Partition Pruning Sink Operator + partition key expr: ds + target column name: ds + target work: Map 1 + Reducer 15 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Filter Operator + predicate: _col0 is not null (type: boolean) + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Spark Partition Pruning Sink Operator + partition key expr: ds + target column name: ds + target work: Map 4 + Reducer 17 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Filter Operator + predicate: _col0 is not null (type: boolean) + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Spark Partition Pruning Sink Operator + partition key expr: ds + target column name: ds + target work: Map 4 + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 2) + Reducer 5 <- Map 4 (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 + Group By Operator + keys: ds (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 + Map 4 + 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 + Group By Operator + keys: ds (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 + 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: ds + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(ds) + 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: ds + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(ds) + 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 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + 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: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + 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 + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Reducer 7 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Filter Operator + predicate: _col0 is not null (type: boolean) + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Reducer 9 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Filter Operator + predicate: _col0 is not null (type: boolean) + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + + 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: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 ds (type: string) + 1 ds (type: string) + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Spark Partition Pruning Sink Operator + partition key expr: ds + 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: 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 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + 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: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 ds (type: string) + 1 ds (type: string) + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Spark Partition Pruning Sink Operator + partition key expr: ds + 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: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 _col3 (type: string) + 1 hr (type: string) + Select Operator + expressions: hr (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Spark Partition Pruning Sink Operator + partition key expr: hr + 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: 550 Data size: 5843 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: 605 Data size: 6427 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 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + 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: 0 Data size: 0 Basic stats: NONE 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 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Spark Partition Pruning Sink Operator + partition key expr: ds + target column name: ds + target work: Map 1 + Select Operator + expressions: hr (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Spark Partition Pruning Sink Operator + partition key expr: hr + 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: 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) + 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 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + 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: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 ds (type: string) + 1 ds (type: string) + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Spark Partition Pruning Sink Operator + partition key expr: ds + 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: 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 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + 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: 0 Data size: 0 Basic stats: NONE 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 + Group By Operator + keys: _col0 (type: double) + mode: hash + outputColumnNames: _col0 + Spark Partition Pruning Sink Operator + partition key expr: UDFToDouble(hr) + 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: 500 Data size: 5312 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: 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) + 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 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + 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: 0 Data size: 0 Basic stats: NONE 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 + Group By Operator + keys: _col0 (type: double) + mode: hash + outputColumnNames: _col0 + Spark Partition Pruning Sink Operator + partition key expr: (hr * 2) + 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: 500 Data size: 5312 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: 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) + 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 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + 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: -- 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-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 5 + Map Operator Tree: + TableScan + alias: srcpart + filterExpr: ds is not null (type: boolean) + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Spark Partition Pruning Sink Operator + partition key expr: ds + target column name: ds + target work: Map 1 + + Stage: Stage-3 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP, 2) +#### 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 + expressions: '2008-04-08' (type: string) + outputColumnNames: ds + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: ds (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 500 Data size: 5312 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: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Reducer 2 + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Group By Operator + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 ds (type: string) + 1 _col0 (type: string) + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Spark Partition Pruning Sink Operator + partition key expr: ds + target column name: ds + target work: Map 3 + + 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 + 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 _col0 (type: string) + input vertices: + 1 Reducer 2 + 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 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 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +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 +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 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 +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 #### +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 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + 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) + Filter Operator + predicate: (date = '2008-04-08') (type: boolean) + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Spark Partition Pruning Sink Operator + partition key expr: ds + 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: 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 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + 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 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + 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: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 ds (type: string) + 1 ds (type: string) + Select Operator + expressions: ds (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Spark Partition Pruning Sink Operator + partition key expr: ds + 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: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 '11' (type: string) + 1 '11' (type: string) + Select Operator + expressions: '11' (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Spark Partition Pruning Sink Operator + partition key expr: hr + 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: 275 Data size: 2921 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: 302 Data size: 3213 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 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + 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-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: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Spark HashTable Sink Operator + keys: + 0 ds (type: string) + 1 ds (type: string) + Local Work: + Map Reduce Local Work + Map 4 + 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: 0 Data size: 0 Basic stats: NONE 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 2 <- Map 1 (GROUP, 1) +#### A masked pattern was here #### + Vertices: + Map 1 + 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 + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + 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 + Select Operator + expressions: ds (type: string) + outputColumnNames: ds + Group By Operator + aggregations: min(ds) + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: string) + Map 8 + Map Operator Tree: + TableScan + alias: srcpart + Select Operator + expressions: ds (type: string) + outputColumnNames: ds + Group By Operator + aggregations: max(ds) + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + sort order: + value expressions: _col0 (type: string) + Reducer 11 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Filter Operator + predicate: _col0 is not null (type: boolean) + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Spark Partition Pruning Sink Operator + partition key expr: ds + target column name: ds + target work: Map 1 + Reducer 9 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Filter Operator + predicate: _col0 is not null (type: boolean) + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Spark Partition Pruning Sink Operator + partition key expr: ds + 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 + 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 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: ds + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: max(ds) + 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: ds + Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: min(ds) + 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 ds (type: string) + 1 _col0 (type: string) + outputColumnNames: _col2 + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col2 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1100 Data size: 11686 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: 1100 Data size: 11686 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: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 5 + Reduce Operator Tree: + Group By Operator + aggregations: max(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Filter Operator + predicate: _col0 is not null (type: boolean) + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Reducer 7 + Reduce Operator Tree: + Group By Operator + aggregations: min(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Filter Operator + predicate: _col0 is not null (type: boolean) + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + + 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