You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jc...@apache.org on 2019/04/08 23:37:57 UTC

[hive] branch master updated: HIVE-20822: Improvements to push computation to JDBC from Calcite (Jesus Camacho Rodriguez, reviewed by Daniel Dai)

This is an automated email from the ASF dual-hosted git repository.

jcamacho pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
     new 877757c  HIVE-20822: Improvements to push computation to JDBC from Calcite (Jesus Camacho Rodriguez, reviewed by Daniel Dai)
877757c is described below

commit 877757cb9fabd666bf21ed902183c5b10d341d73
Author: Jesus Camacho Rodriguez <jc...@apache.org>
AuthorDate: Mon Oct 29 20:55:34 2018 -0700

    HIVE-20822: Improvements to push computation to JDBC from Calcite (Jesus Camacho Rodriguez, reviewed by Daniel Dai)
    
    Close apache/hive#589
---
 .../test/resources/testconfiguration.properties    |    1 +
 .../calcite/reloperators/HiveBetween.java          |   78 +
 .../reloperators/jdbc/HiveJdbcConverter.java       |    2 +-
 .../reloperators/jdbc/JdbcHiveTableScan.java       |    4 +
 .../calcite/rules/jdbc/HiveJdbcImplementor.java    |  251 +-
 .../rules/jdbc/JDBCAggregationPushDownRule.java    |   41 +-
 .../rules/jdbc/JDBCExpandExpressionsRule.java      |  229 +
 .../rules/jdbc/JDBCProjectPushDownRule.java        |   16 +-
 .../calcite/rules/jdbc/JDBCSortPushDownRule.java   |   24 +-
 .../optimizer/calcite/rules/jdbc/package-info.java |   22 -
 .../hadoop/hive/ql/parse/CalcitePlanner.java       |    5 +-
 .../queries/clientpositive/external_jdbc_table2.q  |   36 +
 .../clientpositive/external_jdbc_table_perf.q      | 1806 ++++++
 ql/src/test/results/clientpositive/join34.q.out    |   11 +
 ql/src/test/results/clientpositive/join35.q.out    |   13 +
 .../clientpositive/llap/external_jdbc_table2.q.out |  148 +-
 .../clientpositive/llap/external_jdbc_table4.q.out |   10 +-
 .../llap/external_jdbc_table_perf.q.out            | 6230 ++++++++++++++++++++
 .../results/clientpositive/llap/sharedwork.q.out   |   16 +
 ql/src/test/results/clientpositive/pcr.q.out       |    4 +
 .../clientpositive/rand_partitionpruner3.q.out     |    6 +
 .../test/results/clientpositive/spark/join34.q.out |   11 +
 .../test/results/clientpositive/spark/join35.q.out |   13 +
 ql/src/test/results/clientpositive/spark/pcr.q.out |    4 +
 24 files changed, 8904 insertions(+), 77 deletions(-)

diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index 96dfbc4..d7c84cd 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -530,6 +530,7 @@ minillaplocal.query.files=\
   external_jdbc_table3.q,\
   external_jdbc_table4.q,\
   external_jdbc_table_partition.q,\
+  external_jdbc_table_perf.q,\
   external_jdbc_table_typeconversion.q,\
   fullouter_mapjoin_1_optimized.q,\
   fold_varchar.q,\
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveBetween.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveBetween.java
index ffc705f..03ab2fc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveBetween.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveBetween.java
@@ -19,19 +19,42 @@ package org.apache.hadoop.hive.ql.optimizer.calcite.reloperators;
 
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlCallBinding;
 import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.fun.SqlBetweenOperator;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.type.ReturnTypes;
 import org.apache.calcite.sql.type.SqlOperandTypeInference;
 import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.util.SqlBasicVisitor;
+import org.apache.calcite.util.Util;
 
 public class HiveBetween extends SqlSpecialOperator {
 
   public static final SqlSpecialOperator INSTANCE =
           new HiveBetween();
 
+  /**
+   * Ordinal of the 'value' operand.
+   */
+  public static final int VALUE_OPERAND = 1;
+
+  /**
+   * Ordinal of the 'lower' operand.
+   */
+  public static final int LOWER_OPERAND = 2;
+
+  /**
+   * Ordinal of the 'upper' operand.
+   */
+  public static final int UPPER_OPERAND = 3;
+
   private HiveBetween() {
     super(
         "BETWEEN",
@@ -72,4 +95,59 @@ public class HiveBetween extends SqlSpecialOperator {
           }
         }
       };
+
+  public void unparse(
+      SqlWriter writer,
+      SqlCall call,
+      int leftPrec,
+      int rightPrec) {
+    final SqlWriter.Frame frame =
+        writer.startList(SqlWriter.FrameTypeEnum.create("BETWEEN"), "", "");
+    call.operand(VALUE_OPERAND).unparse(writer, getLeftPrec(), 0);
+    writer.sep(super.getName());
+
+    // If the expression for the lower bound contains a call to an AND
+    // operator, we need to wrap the expression in parentheses to prevent
+    // the AND from associating with BETWEEN. For example, we should
+    // unparse
+    //    a BETWEEN b OR (c AND d) OR e AND f
+    // as
+    //    a BETWEEN (b OR c AND d) OR e) AND f
+    // If it were unparsed as
+    //    a BETWEEN b OR c AND d OR e AND f
+    // then it would be interpreted as
+    //    (a BETWEEN (b OR c) AND d) OR (e AND f)
+    // which would be wrong.
+    final SqlNode lower = call.operand(LOWER_OPERAND);
+    final SqlNode upper = call.operand(UPPER_OPERAND);
+    int lowerPrec = new AndFinder().containsAnd(lower) ? 100 : 0;
+    lower.unparse(writer, lowerPrec, lowerPrec);
+    writer.sep("AND");
+    upper.unparse(writer, 0, getRightPrec());
+    writer.endList(frame);
+  }
+
+  //~ Inner Classes ----------------------------------------------------------
+
+  /**
+   * Finds an AND operator in an expression.
+   */
+  private static class AndFinder extends SqlBasicVisitor<Void> {
+    public Void visit(SqlCall call) {
+      final SqlOperator operator = call.getOperator();
+      if (operator == SqlStdOperatorTable.AND) {
+        throw Util.FoundOne.NULL;
+      }
+      return super.visit(call);
+    }
+
+    boolean containsAnd(SqlNode node) {
+      try {
+        node.accept(this);
+        return false;
+      } catch (Util.FoundOne e) {
+        return true;
+      }
+    }
+  }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/jdbc/HiveJdbcConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/jdbc/HiveJdbcConverter.java
index 3820602..4bedc18 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/jdbc/HiveJdbcConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/jdbc/HiveJdbcConverter.java
@@ -114,7 +114,7 @@ public class HiveJdbcConverter extends ConverterImpl implements HiveRelNode {
           projects, nodeToTranslate.getRowType());
     }
     final HiveJdbcImplementor.Result result =
-        jdbcImplementor.translate(topProject);
+        jdbcImplementor.visit(topProject);
     return result.asStatement().toSqlString(dialect).getSql();
   }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/jdbc/JdbcHiveTableScan.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/jdbc/JdbcHiveTableScan.java
index 5b9a635..a59d58a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/jdbc/JdbcHiveTableScan.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/jdbc/JdbcHiveTableScan.java
@@ -27,6 +27,7 @@ import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelNode;
 
+import org.apache.calcite.rel.RelWriter;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan;
 /**
  * Relational expression representing a scan of a HiveDB collection.
@@ -55,4 +56,7 @@ public class JdbcHiveTableScan extends JdbcTableScan {
     return hiveTableScan;
   }
 
+  @Override public RelWriter explainTerms(RelWriter pw) {
+    return hiveTableScan.explainTerms(pw);
+  }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/jdbc/HiveJdbcImplementor.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/jdbc/HiveJdbcImplementor.java
index 531e769..db955b9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/jdbc/HiveJdbcImplementor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/jdbc/HiveJdbcImplementor.java
@@ -17,17 +17,32 @@
  */
 package org.apache.hadoop.hive.ql.optimizer.calcite.rules.jdbc;
 
+import com.google.common.collect.ImmutableList;
 import java.util.ArrayList;
 import java.util.List;
 import org.apache.calcite.adapter.java.JavaTypeFactory;
 import org.apache.calcite.adapter.jdbc.JdbcImplementor;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.CorrelationId;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.JoinConditionType;
+import org.apache.calcite.sql.JoinType;
 import org.apache.calcite.sql.SqlDialect;
+import org.apache.calcite.sql.SqlJoin;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 
 public class HiveJdbcImplementor extends JdbcImplementor {
 
@@ -35,9 +50,8 @@ public class HiveJdbcImplementor extends JdbcImplementor {
     super(dialect, typeFactory);
   }
 
-  public Result translate(Project e) {
-    // This variant is for the top project as we want to keep
-    // the column aliases instead of producing STAR
+  @Override public Result visit(Project e) {
+    // This variant keeps the column aliases instead of producing STAR
     Result x = visitChild(0, e.getInput());
     parseCorrelTable(e, x);
     final Builder builder =
@@ -52,9 +66,240 @@ public class HiveJdbcImplementor extends JdbcImplementor {
     return builder.result();
   }
 
+  @Override public Result visit(Sort e) {
+    Result x = visitChild(0, e.getInput());
+    Builder builder = x.builder(e, Clause.ORDER_BY);
+    List<SqlNode> orderByList = Expressions.list();
+    for (RelFieldCollation field : e.getCollation().getFieldCollations()) {
+      builder.addOrderItem(orderByList, field);
+    }
+
+    // Create select list as we want to keep the column aliases
+    // instead of producing STAR
+    final List<SqlNode> selectList = new ArrayList<>();
+    for (int i = 0; i < e.getRowType().getFieldCount(); i++) {
+      RexInputRef ref = RexInputRef.of(i, e.getRowType());
+      SqlNode sqlExpr = builder.context.toSql(null, ref);
+      addSelect(selectList, sqlExpr, e.getRowType());
+    }
+    builder.setSelect(new SqlNodeList(selectList, POS));
+
+    if (!orderByList.isEmpty()) {
+      builder.setOrderBy(new SqlNodeList(orderByList, POS));
+      x = builder.result();
+    }
+    if (e.fetch != null) {
+      builder = x.builder(e, Clause.FETCH);
+      builder.setFetch(builder.context.toSql(null, e.fetch));
+      x = builder.result();
+    }
+    if (e.offset != null) {
+      builder = x.builder(e, Clause.OFFSET);
+      builder.setOffset(builder.context.toSql(null, e.offset));
+      x = builder.result();
+    }
+    return x;
+  }
+
+  @Override public Result visit(Join e) {
+    final Result leftResult = visitChild(0, e.getLeft()).resetAlias();
+    final Result rightResult = visitChild(1, e.getRight()).resetAlias();
+    final Context leftContext = leftResult.qualifiedContext();
+    final Context rightContext = rightResult.qualifiedContext();
+    SqlNode sqlCondition = null;
+    SqlLiteral condType = JoinConditionType.ON.symbol(POS);
+    JoinType joinType = joinType(e.getJoinType());
+    if (e.getJoinType() == JoinRelType.INNER && e.getCondition().isAlwaysTrue()) {
+      joinType = JoinType.COMMA;
+      condType = JoinConditionType.NONE.symbol(POS);
+    } else {
+      sqlCondition = convertConditionToSqlNode(e.getCondition(),
+          leftContext,
+          rightContext,
+          e.getLeft().getRowType().getFieldCount());
+    }
+    SqlNode join =
+        new SqlJoin(POS,
+            leftResult.asFrom(),
+            SqlLiteral.createBoolean(false, POS),
+            joinType.symbol(POS),
+            rightResult.asFrom(),
+            condType,
+            sqlCondition);
+    return result(join, leftResult, rightResult);
+  }
+
+  /**
+   * Converts a {@link RexNode} condition into a {@link SqlNode}.
+   *
+   * @param node            Join condition
+   * @param leftContext     Left context
+   * @param rightContext    Right context
+   * @param leftFieldCount  Number of fields on left result
+   * @return SqlNode that represents the condition
+   */
+  public static SqlNode convertConditionToSqlNode(RexNode node,
+      Context leftContext, Context rightContext, int leftFieldCount) {
+    // This variant replaces the default case for a call so it does not
+    // throw an Exception
+    if (node.isAlwaysTrue()) {
+      return SqlLiteral.createBoolean(true, POS);
+    }
+    if (node.isAlwaysFalse()) {
+      return SqlLiteral.createBoolean(false, POS);
+    }
+    if (node instanceof RexInputRef) {
+      Context joinContext = leftContext.implementor().joinContext(leftContext, rightContext);
+      return joinContext.toSql(null, node);
+    }
+    if (!(node instanceof RexCall)) {
+      throw new AssertionError(node);
+    }
+    final List<RexNode> operands;
+    final SqlOperator op;
+    final Context joinContext;
+    switch (node.getKind()) {
+    case AND:
+    case OR:
+      operands = ((RexCall) node).getOperands();
+      op = ((RexCall) node).getOperator();
+      SqlNode sqlCondition = null;
+      for (RexNode operand : operands) {
+        SqlNode x = convertConditionToSqlNode(operand, leftContext,
+            rightContext, leftFieldCount);
+        if (sqlCondition == null) {
+          sqlCondition = x;
+        } else {
+          sqlCondition = op.createCall(POS, sqlCondition, x);
+        }
+      }
+      return sqlCondition;
+
+    case EQUALS:
+    case IS_NOT_DISTINCT_FROM:
+    case NOT_EQUALS:
+    case GREATER_THAN:
+    case GREATER_THAN_OR_EQUAL:
+    case LESS_THAN:
+    case LESS_THAN_OR_EQUAL:
+      node = stripCastFromString(node);
+      operands = ((RexCall) node).getOperands();
+      op = ((RexCall) node).getOperator();
+      if (operands.size() == 2
+          && operands.get(0) instanceof RexInputRef
+          && operands.get(1) instanceof RexInputRef) {
+        final RexInputRef op0 = (RexInputRef) operands.get(0);
+        final RexInputRef op1 = (RexInputRef) operands.get(1);
+
+        if (op0.getIndex() < leftFieldCount
+            && op1.getIndex() >= leftFieldCount) {
+          // Arguments were of form 'op0 = op1'
+          return op.createCall(POS,
+              leftContext.field(op0.getIndex()),
+              rightContext.field(op1.getIndex() - leftFieldCount));
+        }
+        if (op1.getIndex() < leftFieldCount
+            && op0.getIndex() >= leftFieldCount) {
+          // Arguments were of form 'op1 = op0'
+          return reverseOperatorDirection(op).createCall(POS,
+              leftContext.field(op1.getIndex()),
+              rightContext.field(op0.getIndex() - leftFieldCount));
+        }
+      }
+      joinContext =
+          leftContext.implementor().joinContext(leftContext, rightContext);
+      return joinContext.toSql(null, node);
+
+    case IS_NULL:
+    case IS_NOT_NULL:
+      operands = ((RexCall) node).getOperands();
+      if (operands.size() == 1
+          && operands.get(0) instanceof RexInputRef) {
+        op = ((RexCall) node).getOperator();
+        final RexInputRef op0 = (RexInputRef) operands.get(0);
+        if (op0.getIndex() < leftFieldCount) {
+          return op.createCall(POS, leftContext.field(op0.getIndex()));
+        } else {
+          return op.createCall(POS,
+              rightContext.field(op0.getIndex() - leftFieldCount));
+        }
+      }
+      joinContext =
+          leftContext.implementor().joinContext(leftContext, rightContext);
+      return joinContext.toSql(null, node);
+
+    default:
+      joinContext =
+          leftContext.implementor().joinContext(leftContext, rightContext);
+      return joinContext.toSql(null, node);
+    }
+  }
+
+  /** Removes cast from string.
+   *
+   * <p>For example, {@code x > CAST('2015-01-07' AS DATE)}
+   * becomes {@code x > '2015-01-07'}.
+   */
+  private static RexNode stripCastFromString(RexNode node) {
+    switch (node.getKind()) {
+    case EQUALS:
+    case IS_NOT_DISTINCT_FROM:
+    case NOT_EQUALS:
+    case GREATER_THAN:
+    case GREATER_THAN_OR_EQUAL:
+    case LESS_THAN:
+    case LESS_THAN_OR_EQUAL:
+      final RexCall call = (RexCall) node;
+      final RexNode o0 = call.operands.get(0);
+      final RexNode o1 = call.operands.get(1);
+      if (o0.getKind() == SqlKind.CAST
+          && o1.getKind() != SqlKind.CAST) {
+        final RexNode o0b = ((RexCall) o0).getOperands().get(0);
+        switch (o0b.getType().getSqlTypeName()) {
+        case CHAR:
+        case VARCHAR:
+          return call.clone(call.getType(), ImmutableList.of(o0b, o1));
+        }
+      }
+      if (o1.getKind() == SqlKind.CAST
+          && o0.getKind() != SqlKind.CAST) {
+        final RexNode o1b = ((RexCall) o1).getOperands().get(0);
+        switch (o1b.getType().getSqlTypeName()) {
+        case CHAR:
+        case VARCHAR:
+          return call.clone(call.getType(), ImmutableList.of(o0, o1b));
+        }
+      }
+    }
+    return node;
+  }
+
+  private static SqlOperator reverseOperatorDirection(SqlOperator op) {
+    switch (op.kind) {
+    case GREATER_THAN:
+      return SqlStdOperatorTable.LESS_THAN;
+    case GREATER_THAN_OR_EQUAL:
+      return SqlStdOperatorTable.LESS_THAN_OR_EQUAL;
+    case LESS_THAN:
+      return SqlStdOperatorTable.GREATER_THAN;
+    case LESS_THAN_OR_EQUAL:
+      return SqlStdOperatorTable.GREATER_THAN_OR_EQUAL;
+    case EQUALS:
+    case IS_NOT_DISTINCT_FROM:
+    case NOT_EQUALS:
+      return op;
+    default:
+      throw new AssertionError(op);
+    }
+  }
+
   private void parseCorrelTable(RelNode relNode, Result x) {
     for (CorrelationId id : relNode.getVariablesSet()) {
       correlTableMap.put(id, x.qualifiedContext());
     }
   }
+
+  @Override public Result implement(RelNode node) {
+    return dispatch(node);
+  }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/jdbc/JDBCAggregationPushDownRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/jdbc/JDBCAggregationPushDownRule.java
index c51ae0d..e219639 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/jdbc/JDBCAggregationPushDownRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/jdbc/JDBCAggregationPushDownRule.java
@@ -17,14 +17,11 @@
  */
 package org.apache.hadoop.hive.ql.optimizer.calcite.rules.jdbc;
 
-import java.util.Arrays;
-
 import org.apache.calcite.adapter.jdbc.JdbcRules.JdbcAggregate;
-import org.apache.calcite.adapter.jdbc.JdbcRules.JdbcAggregateRule;
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.InvalidRelException;
+import org.apache.calcite.rel.core.Aggregate.Group;
 import org.apache.calcite.rel.core.AggregateCall;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlKind;
@@ -56,6 +53,11 @@ public class JDBCAggregationPushDownRule extends RelOptRule {
     final HiveAggregate agg = call.rel(0);
     final HiveJdbcConverter converter = call.rel(1);
 
+    if (agg.getGroupType() != Group.SIMPLE) {
+      // TODO: Grouping sets not supported yet
+      return false;
+    }
+
     for (AggregateCall relOptRuleOperand : agg.getAggCallList()) {
       SqlAggFunction f = relOptRuleOperand.getAggregation();
       if (f instanceof HiveSqlCountAggFunction) {
@@ -75,20 +77,27 @@ public class JDBCAggregationPushDownRule extends RelOptRule {
 
   @Override
   public void onMatch(RelOptRuleCall call) {
-    LOG.debug("MyAggregationPushDownRule.onMatch has been called");
+    LOG.debug("JDBCAggregationPushDownRule has been called");
 
-    final HiveAggregate agg = call.rel(0);
+    final HiveAggregate aggregate = call.rel(0);
     final HiveJdbcConverter converter = call.rel(1);
 
-    Aggregate newHiveAggregate = agg.copy(agg.getTraitSet(), converter.getInput(),
-            agg.getIndicatorCount() !=0, agg.getGroupSet(), agg.getGroupSets(), agg.getAggCallList());
-    JdbcAggregate newJdbcAggregate =
-            (JdbcAggregate) new JdbcAggregateRule(converter.getJdbcConvention()).convert(newHiveAggregate);
-    if (newJdbcAggregate != null) {
-      RelNode converterRes = converter.copy(converter.getTraitSet(), Arrays.asList(newJdbcAggregate));
-
-      call.transformTo(converterRes);
+    JdbcAggregate jdbcAggregate;
+    try {
+      jdbcAggregate = new JdbcAggregate(
+          aggregate.getCluster(),
+          aggregate.getTraitSet().replace(converter.getJdbcConvention()),
+          converter.getInput(),
+          aggregate.indicator,
+          aggregate.getGroupSet(),
+          aggregate.getGroupSets(),
+          aggregate.getAggCallList());
+    } catch (InvalidRelException e) {
+      LOG.warn(e.toString());
+      return;
     }
+
+    call.transformTo(converter.copy(converter.getTraitSet(), jdbcAggregate));
   }
 
-};
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/jdbc/JDBCExpandExpressionsRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/jdbc/JDBCExpandExpressionsRule.java
new file mode 100644
index 0000000..1c8a100
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/jdbc/JDBCExpandExpressionsRule.java
@@ -0,0 +1,229 @@
+/*
+ * 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.calcite.rules.jdbc;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.calcite.adapter.jdbc.JdbcRules.JdbcFilter;
+import org.apache.calcite.adapter.jdbc.JdbcRules.JdbcJoin;
+import org.apache.calcite.adapter.jdbc.JdbcRules.JdbcProject;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * JDBCExpandExpressionsRule that rewrites IN clauses over struct operators
+ * into OR/AND expressions.
+ */
+
+public abstract class JDBCExpandExpressionsRule extends RelOptRule {
+  private static final Logger LOG = LoggerFactory.getLogger(JDBCExpandExpressionsRule.class);
+
+  public static final JDBCExpandExpressionsRule.FilterCondition FILTER_INSTANCE =
+      new JDBCExpandExpressionsRule.FilterCondition();
+  public static final JDBCExpandExpressionsRule.JoinCondition JOIN_INSTANCE =
+      new JDBCExpandExpressionsRule.JoinCondition();
+  public static final JDBCExpandExpressionsRule.ProjectionExpressions PROJECT_INSTANCE =
+      new JDBCExpandExpressionsRule.ProjectionExpressions();
+
+
+  private JDBCExpandExpressionsRule(RelOptRuleOperand operand) {
+    super(operand);
+  }
+
+  /** Rule adapter to apply the transformation to Filter conditions. */
+  private static class FilterCondition extends JDBCExpandExpressionsRule {
+
+    private FilterCondition() {
+      super(operand(JdbcFilter.class, any()));
+    }
+
+    @Override
+    public void onMatch(RelOptRuleCall call) {
+      LOG.debug("JDBCExpandExpressionsRule.FilterCondition has been called");
+
+      final JdbcFilter filter = call.rel(0);
+      final RexNode condition = filter.getCondition();
+
+      RexNode newCondition = analyzeRexNode(
+          filter.getCluster().getRexBuilder(), condition);
+
+      // If we could not transform anything, we bail out
+      if (newCondition.toString().equals(condition.toString())) {
+        return;
+      }
+
+      RelNode newNode = filter.copy(filter.getTraitSet(), filter.getInput(), newCondition);
+      call.transformTo(newNode);
+    }
+  }
+
+  /** Rule adapter to apply the transformation to Join conditions. */
+  private static class JoinCondition extends JDBCExpandExpressionsRule {
+
+    private JoinCondition () {
+      super(operand(JdbcJoin.class, any()));
+    }
+
+    @Override
+    public void onMatch(RelOptRuleCall call) {
+      LOG.debug("JDBCExpandExpressionsRule.JoinCondition has been called");
+
+      final Join join = call.rel(0);
+      final RexNode condition = RexUtil.pullFactors(
+          join.getCluster().getRexBuilder(), join.getCondition());
+
+      RexNode newCondition = analyzeRexNode(
+          join.getCluster().getRexBuilder(), condition);
+
+      // If we could not transform anything, we bail out
+      if (newCondition.toString().equals(condition.toString())) {
+        return;
+      }
+
+      RelNode newNode = join.copy(join.getTraitSet(),
+          newCondition,
+          join.getLeft(),
+          join.getRight(),
+          join.getJoinType(),
+          join.isSemiJoinDone());
+      call.transformTo(newNode);
+    }
+  }
+
+  /** Rule adapter to apply the transformation to Projections. */
+  private static class ProjectionExpressions extends JDBCExpandExpressionsRule {
+
+    private ProjectionExpressions() {
+      super(operand(JdbcProject.class, any()));
+    }
+
+    @Override
+    public void onMatch(RelOptRuleCall call) {
+      LOG.debug("JDBCExpandExpressionsRule.ProjectionExpressions has been called");
+
+      final Project project = call.rel(0);
+      final RexBuilder rexBuilder = project.getCluster().getRexBuilder();
+      boolean changed = false;
+      List<RexNode> newProjects = new ArrayList<>();
+      for (RexNode oldNode : project.getProjects()) {
+        RexNode newNode = analyzeRexNode(rexBuilder, oldNode);
+        if (!newNode.toString().equals(oldNode.toString())) {
+          changed = true;
+          newProjects.add(newNode);
+        } else {
+          newProjects.add(oldNode);
+        }
+      }
+
+      if (!changed) {
+        return;
+      }
+
+      Project newProject = project.copy(
+          project.getTraitSet(),
+          project.getInput(),
+          newProjects,
+          project.getRowType());
+      call.transformTo(newProject);
+    }
+
+  }
+
+  RexNode analyzeRexNode(RexBuilder rexBuilder, RexNode condition) {
+    RexTransformIntoOrAndClause transformIntoInClause = new RexTransformIntoOrAndClause(rexBuilder);
+    RexNode newCondition = transformIntoInClause.apply(condition);
+    return newCondition;
+  }
+
+  /**
+   * Transforms IN clauses into OR/AND clauses, when possible.
+   */
+  protected static class RexTransformIntoOrAndClause extends RexShuttle {
+    private final RexBuilder rexBuilder;
+
+    RexTransformIntoOrAndClause(RexBuilder rexBuilder) {
+      this.rexBuilder = rexBuilder;
+    }
+
+    @Override
+    public RexNode visitCall(RexCall inputCall) {
+      RexNode node = super.visitCall(inputCall);
+      if (node instanceof RexCall) {
+        RexCall call = (RexCall) node;
+        switch (call.getKind()) {
+          case IN:
+            return transformIntoOrAndClause(rexBuilder, call);
+          default:
+            break;
+        }
+      }
+      return node;
+    }
+
+    private RexNode transformIntoOrAndClause(RexBuilder rexBuilder, RexCall expression) {
+      assert expression.getKind() == SqlKind.IN;
+
+      if (expression.getOperands().get(0).getKind() != SqlKind.ROW) {
+        // Nothing to do, return expression
+        return expression;
+      }
+
+      final List<RexNode> disjuncts = new ArrayList<>(expression.getOperands().size() - 2);
+      final RexCall columnExpressions = (RexCall) expression.getOperands().get(0);
+      for (int i = 1; i < expression.getOperands().size(); i++) {
+        List<RexNode> conjuncts = new ArrayList<>(columnExpressions.getOperands().size() - 1);
+        RexCall valueExpressions = (RexCall) expression.getOperands().get(i);
+        for (int j = 0; j < columnExpressions.getOperands().size(); j++) {
+          conjuncts.add(rexBuilder.makeCall(
+              SqlStdOperatorTable.EQUALS,
+              columnExpressions.getOperands().get(j),
+              valueExpressions.getOperands().get(j)));
+        }
+        if (conjuncts.size() > 1) {
+          disjuncts.add(rexBuilder.makeCall(
+              SqlStdOperatorTable.AND,
+              conjuncts));
+        } else {
+          disjuncts.add(conjuncts.get(0));
+        }
+      }
+
+      if (disjuncts.size() > 1) {
+        return rexBuilder.makeCall(SqlStdOperatorTable.OR, disjuncts);
+      } else {
+        return disjuncts.get(0);
+      }
+    }
+
+  }
+
+
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/jdbc/JDBCProjectPushDownRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/jdbc/JDBCProjectPushDownRule.java
index 0e88f53..920518a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/jdbc/JDBCProjectPushDownRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/jdbc/JDBCProjectPushDownRule.java
@@ -68,14 +68,14 @@ public class JDBCProjectPushDownRule extends RelOptRule {
     final HiveProject project = call.rel(0);
     final HiveJdbcConverter converter = call.rel(1);
 
-    Project newHiveProject = project.copy(project.getTraitSet(), converter.getInput(),
-            project.getProjects(), project.getRowType());
-    JdbcProject newJdbcProject =
-            (JdbcProject) new JdbcProjectRule(converter.getJdbcConvention()).convert(newHiveProject);
-    if (newJdbcProject != null) {
-      RelNode converterRes = converter.copy(converter.getTraitSet(), Arrays.asList(newJdbcProject));
-      call.transformTo(converterRes);
-    }
+    JdbcProject jdbcProject = new JdbcProject(
+        project.getCluster(),
+        project.getTraitSet().replace(converter.getJdbcConvention()),
+        converter.getInput(),
+        project.getProjects(),
+        project.getRowType());
+
+    call.transformTo(converter.copy(converter.getTraitSet(), jdbcProject));
   }
 
 };
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/jdbc/JDBCSortPushDownRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/jdbc/JDBCSortPushDownRule.java
index 33dc280..79c6c12 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/jdbc/JDBCSortPushDownRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/jdbc/JDBCSortPushDownRule.java
@@ -45,11 +45,11 @@ public class JDBCSortPushDownRule extends RelOptRule {
 
   public JDBCSortPushDownRule() {
     super(operand(HiveSortLimit.class,
-        operand(HiveJdbcConverter.class, operand(RelNode.class, any()))));
+        operand(HiveJdbcConverter.class, any())));
   }
 
   public boolean matches(RelOptRuleCall call) {
-    final Sort sort = (Sort) call.rel(0);
+    final Sort sort = call.rel(0);
     final HiveJdbcConverter conv = call.rel(1);
 
     for (RexNode currCall : sort.getChildExps()) {
@@ -67,18 +67,16 @@ public class JDBCSortPushDownRule extends RelOptRule {
 
     final HiveSortLimit sort = call.rel(0);
     final HiveJdbcConverter converter = call.rel(1);
-    final RelNode input = call.rel(2);
 
-    Sort newHiveSort = sort.copy(sort.getTraitSet(), input, sort.getCollation(), sort.getOffsetExpr(),
-            sort.getFetchExpr());
+    JdbcSort jdbcSort = new JdbcSort(
+        sort.getCluster(),
+        sort.getTraitSet().replace(converter.getJdbcConvention()),
+        converter.getInput(),
+        sort.getCollation(),
+        sort.offset,
+        sort.fetch);
 
-    JdbcSort newJdbcSort =
-            (JdbcSort) new JdbcSortRule(converter.getJdbcConvention()).convert(newHiveSort, false);
-    if (newJdbcSort != null) {
-      RelNode converterRes = converter.copy(converter.getTraitSet(), Arrays.asList(newJdbcSort));
-
-      call.transformTo(converterRes);
-    }
+    call.transformTo(converter.copy(converter.getTraitSet(), jdbcSort));
   }
 
-};
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/jdbc/package-info.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/jdbc/package-info.java
deleted file mode 100644
index 08a17aa..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/jdbc/package-info.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * 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.
- */
-
-/**
- * JDBC external table rules that pushes down operators as much as possible to the external jdbc table.
- */
-package org.apache.hadoop.hive.ql.optimizer.calcite.rules.jdbc;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
index 8042f60..20501cf 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
@@ -225,6 +225,7 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveUnionPullUpConstant
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveWindowingFixRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.jdbc.JDBCAbstractSplitFilterRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.jdbc.JDBCAggregationPushDownRule;
+import org.apache.hadoop.hive.ql.optimizer.calcite.rules.jdbc.JDBCExpandExpressionsRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.jdbc.JDBCExtractJoinFilterRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.jdbc.JDBCFilterJoinRule;
 import org.apache.hadoop.hive.ql.optimizer.calcite.rules.jdbc.JDBCFilterPushDownRule;
@@ -1969,6 +1970,9 @@ public class CalcitePlanner extends SemanticAnalyzer {
 
       if (conf.getBoolVar(ConfVars.HIVE_ENABLE_JDBC_PUSHDOWN)) {
         List<RelOptRule> rules = Lists.newArrayList();
+        rules.add(JDBCExpandExpressionsRule.FILTER_INSTANCE);
+        rules.add(JDBCExpandExpressionsRule.JOIN_INSTANCE);
+        rules.add(JDBCExpandExpressionsRule.PROJECT_INSTANCE);
         rules.add(JDBCExtractJoinFilterRule.INSTANCE);
         rules.add(JDBCAbstractSplitFilterRule.SPLIT_FILTER_ABOVE_JOIN);
         rules.add(JDBCAbstractSplitFilterRule.SPLIT_FILTER_ABOVE_CONVERTER);
@@ -3001,7 +3005,6 @@ public class CalcitePlanner extends SemanticAnalyzer {
             final String url = tabMetaData.getProperty(Constants.JDBC_URL);
             final String driver = tabMetaData.getProperty(Constants.JDBC_DRIVER);
             final String user = tabMetaData.getProperty(Constants.JDBC_USERNAME);
-            //final String query = tabMetaData.getProperty("hive.sql.query");
             String pswd = tabMetaData.getProperty(Constants.JDBC_PASSWORD);
             if (pswd == null) {
               String keystore = tabMetaData.getProperty(Constants.JDBC_KEYSTORE);
diff --git a/ql/src/test/queries/clientpositive/external_jdbc_table2.q b/ql/src/test/queries/clientpositive/external_jdbc_table2.q
index 28a2180..b8c6cc4 100644
--- a/ql/src/test/queries/clientpositive/external_jdbc_table2.q
+++ b/ql/src/test/queries/clientpositive/external_jdbc_table2.q
@@ -134,6 +134,42 @@ SELECT * FROM db1_ext_auth1 UNION ALL SELECT * FROM db1_ext_auth2;
 
 SELECT * FROM db1_ext_auth1 UNION ALL SELECT * FROM db1_ext_auth2;
 
+EXPLAIN
+SELECT P.ikey
+FROM
+  db1_ext_auth1 P
+WHERE
+  NOT TRUE OR P.`ikey` IS NOT NULL
+  AND (P.`bkey` = 10 AND P.`dkey` = 15.15
+    OR P.`bkey` = 20 AND P.`dkey` = 25.25);
+
+SELECT P.ikey
+FROM
+  db1_ext_auth1 P
+WHERE
+  NOT TRUE OR P.`ikey` IS NOT NULL
+  AND (P.`bkey` = 10 AND P.`dkey` = 15.15
+    OR P.`bkey` = 20 AND P.`dkey` = 25.25);
+    
+EXPLAIN
+SELECT P.ikey
+FROM
+  db1_ext_auth1 P
+WHERE
+  NOT TRUE OR P.`ikey` IS NOT NULL
+  AND (P.`bkey` = 10 AND P.`dkey` = 15.15
+    OR P.`bkey` = 20 AND P.`dkey` = 25.25)
+  AND fkey = null;
+
+SELECT P.ikey
+FROM
+  db1_ext_auth1 P
+WHERE
+  NOT TRUE OR P.`ikey` IS NOT NULL
+  AND (P.`bkey` = 10 AND P.`dkey` = 15.15
+    OR P.`bkey` = 20 AND P.`dkey` = 25.25)
+  AND fkey = null;
+
 set hive.jdbc.pushdown.enable=false;
 
 EXPLAIN
diff --git a/ql/src/test/queries/clientpositive/external_jdbc_table_perf.q b/ql/src/test/queries/clientpositive/external_jdbc_table_perf.q
new file mode 100644
index 0000000..a3834f6
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/external_jdbc_table_perf.q
@@ -0,0 +1,1806 @@
+--! qt:dataset:src
+
+CREATE TEMPORARY FUNCTION dboutput AS 'org.apache.hadoop.hive.contrib.genericudf.example.GenericUDFDBOutput';
+
+FROM src
+SELECT
+dboutput ('jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf;create=true','user1','passwd1',
+'CREATE TABLE CATALOG_SALES ("cs_sold_date_sk" INTEGER, "cs_sold_time_sk" INTEGER, "cs_ship_date_sk" INTEGER, "cs_bill_customer_sk" INTEGER,
+  "cs_bill_cdemo_sk" INTEGER, "cs_bill_hdemo_sk" INTEGER, "cs_bill_addr_sk" INTEGER, "cs_ship_customer_sk" INTEGER, "cs_ship_cdemo_sk" INTEGER,
+  "cs_ship_hdemo_sk" INTEGER, "cs_ship_addr_sk" INTEGER, "cs_call_center_sk" INTEGER, "cs_catalog_page_sk" INTEGER, "cs_ship_mode_sk" INTEGER,
+  "cs_warehouse_sk" INTEGER, "cs_item_sk" INTEGER, "cs_promo_sk" INTEGER, "cs_order_number" INTEGER, "cs_quantity" INTEGER, "cs_wholesale_cost" DECIMAL(7,2),
+  "cs_list_price" DECIMAL(7,2), "cs_sales_price" DECIMAL(7,2), "cs_ext_discount_amt" DECIMAL(7,2), "cs_ext_sales_price" DECIMAL(7,2),
+  "cs_ext_wholesale_cost" DECIMAL(7,2), "cs_ext_list_price" DECIMAL(7,2), "cs_ext_tax" DECIMAL(7,2), "cs_coupon_amt" DECIMAL(7,2),
+  "cs_ext_ship_cost" DECIMAL(7,2), "cs_net_paid" DECIMAL(7,2), "cs_net_paid_inc_tax" DECIMAL(7,2), "cs_net_paid_inc_ship" DECIMAL(7,2),
+  "cs_net_paid_inc_ship_tax" DECIMAL(7,2), "cs_net_profit" DECIMAL(7,2))' ),
+dboutput ('jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf;create=true','user1','passwd1',
+'CREATE TABLE CATALOG_RETURNS ("cr_returned_date_sk" INTEGER, "cr_returned_time_sk" INTEGER, "cr_item_sk" INTEGER, "cr_refunded_customer_sk" INTEGER,
+  "cr_refunded_cdemo_sk" INTEGER, "cr_refunded_hdemo_sk" INTEGER, "cr_refunded_addr_sk" INTEGER, "cr_returning_customer_sk" INTEGER,
+  "cr_returning_cdemo_sk" INTEGER, "cr_returning_hdemo_sk" INTEGER, "cr_returning_addr_sk" INTEGER, "cr_call_center_sk" INTEGER,
+  "cr_catalog_page_sk" INTEGER, "cr_ship_mode_sk" INTEGER, "cr_warehouse_sk" INTEGER, "cr_reason_sk" INTEGER, "cr_order_number" INTEGER,
+  "cr_return_quantity" INTEGER, "cr_return_amount" DECIMAL(7,2), "cr_return_tax" DECIMAL(7,2), "cr_return_amt_inc_tax" DECIMAL(7,2),
+  "cr_fee" DECIMAL(7,2), "cr_return_ship_cost" DECIMAL(7,2), "cr_refunded_cash" DECIMAL(7,2), "cr_reversed_charge" DECIMAL(7,2),
+  "cr_store_credit" DECIMAL(7,2), "cr_net_loss" DECIMAL(7,2))' ),
+dboutput ('jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf;create=true','user1','passwd1',
+'CREATE TABLE STORE_SALES ("ss_sold_date_sk" INTEGER, "ss_sold_time_sk" INTEGER, "ss_item_sk" INTEGER, "ss_customer_sk" INTEGER, "ss_cdemo_sk" INTEGER,
+  "ss_hdemo_sk" INTEGER, "ss_addr_sk" INTEGER, "ss_store_sk" INTEGER, "ss_promo_sk" INTEGER, "ss_ticket_number" INTEGER, "ss_quantity" INTEGER,
+  "ss_wholesale_cost" DECIMAL(7,2), "ss_list_price" DECIMAL(7,2), "ss_sales_price" DECIMAL(7,2), "ss_ext_discount_amt" DECIMAL(7,2),
+  "ss_ext_sales_price" DECIMAL(7,2), "ss_ext_wholesale_cost" DECIMAL(7,2), "ss_ext_list_price" DECIMAL(7,2), "ss_ext_tax" DECIMAL(7,2),
+  "ss_coupon_amt" DECIMAL(7,2), "ss_net_paid" DECIMAL(7,2), "ss_net_paid_inc_tax" DECIMAL(7,2), "ss_net_profit" DECIMAL(7,2))' ),
+dboutput ('jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf;create=true','user1','passwd1',
+'CREATE TABLE STORE_RETURNS ("sr_returned_date_sk" INTEGER, "sr_return_time_sk" INTEGER, "sr_item_sk" INTEGER, "sr_customer_sk" INTEGER,
+  "sr_cdemo_sk" INTEGER, "sr_hdemo_sk" INTEGER, "sr_addr_sk" INTEGER, "sr_store_sk" INTEGER, "sr_reason_sk" INTEGER, "sr_ticket_number" INTEGER,
+  "sr_return_quantity" INTEGER, "sr_return_amt" DECIMAL(7,2), "sr_return_tax" DECIMAL(7,2), "sr_return_amt_inc_tax" DECIMAL(7,2),
+  "sr_fee" DECIMAL(7,2), "sr_return_ship_cost" DECIMAL(7,2), "sr_refunded_cash" DECIMAL(7,2), "sr_reversed_charge" DECIMAL(7,2),
+  "sr_store_credit" DECIMAL(7,2), "sr_net_loss" DECIMAL(7,2))' ),
+dboutput ('jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf;create=true','user1','passwd1',
+'CREATE TABLE WEB_SALES ("ws_sold_date_sk" INTEGER, "ws_sold_time_sk" INTEGER, "ws_ship_date_sk" INTEGER, "ws_item_sk" INTEGER,
+  "ws_bill_customer_sk" INTEGER, "ws_bill_cdemo_sk" INTEGER, "ws_bill_hdemo_sk" INTEGER, "ws_bill_addr_sk" INTEGER, "ws_ship_customer_sk" INTEGER,
+  "ws_ship_cdemo_sk" INTEGER, "ws_ship_hdemo_sk" INTEGER, "ws_ship_addr_sk" INTEGER, "ws_web_page_sk" INTEGER, "ws_web_site_sk" INTEGER,
+  "ws_ship_mode_sk" INTEGER, "ws_warehouse_sk" INTEGER, "ws_promo_sk" INTEGER, "ws_order_number" INTEGER, "ws_quantity" INTEGER,
+  "ws_wholesale_cost" DECIMAL(7,2), "ws_list_price" DECIMAL(7,2), "ws_sales_price" DECIMAL(7,2), "ws_ext_discount_amt" DECIMAL(7,2),
+  "ws_ext_sales_price" DECIMAL(7,2), "ws_ext_wholesale_cost" DECIMAL(7,2), "ws_ext_list_price" DECIMAL(7,2), "ws_ext_tax" DECIMAL(7,2),
+  "ws_coupon_amt" DECIMAL(7,2), "ws_ext_ship_cost" DECIMAL(7,2), "ws_net_paid" DECIMAL(7,2), "ws_net_paid_inc_tax" DECIMAL(7,2),
+  "ws_net_paid_inc_ship" DECIMAL(7,2), "ws_net_paid_inc_ship_tax" DECIMAL(7,2), "ws_net_profit" DECIMAL(7,2))' ),
+dboutput ('jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf;create=true','user1','passwd1',
+'CREATE TABLE WEB_RETURNS ("wr_returned_date_sk" INTEGER, "wr_returned_time_sk" INTEGER, "wr_item_sk" INTEGER, "wr_refunded_customer_sk" INTEGER,
+  "wr_refunded_cdemo_sk" INTEGER, "wr_refunded_hdemo_sk" INTEGER, "wr_refunded_addr_sk" INTEGER, "wr_returning_customer_sk" INTEGER,
+  "wr_returning_cdemo_sk" INTEGER, "wr_returning_hdemo_sk" INTEGER, "wr_returning_addr_sk" INTEGER, "wr_web_page_sk" INTEGER,
+  "wr_reason_sk" INTEGER, "wr_order_number" INTEGER, "wr_return_quantity" INTEGER, "wr_return_amt" DECIMAL(7,2), "wr_return_tax" DECIMAL(7,2),
+  "wr_return_amt_inc_tax" DECIMAL(7,2), "wr_fee" DECIMAL(7,2), "wr_return_ship_cost" DECIMAL(7,2), "wr_refunded_cash" DECIMAL(7,2),
+  "wr_reversed_charge" DECIMAL(7,2), "wr_account_credit" DECIMAL(7,2), "wr_net_loss" DECIMAL(7,2))' ),
+dboutput ('jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf;create=true','user1','passwd1',
+'CREATE TABLE CUSTOMER ("c_customer_sk" INTEGER, "c_customer_id" CHAR(16), "c_current_cdemo_sk" INTEGER, "c_current_hdemo_sk" INTEGER,
+  "c_current_addr_sk" INTEGER, "c_first_shipto_date_sk" INTEGER, "c_first_sales_date_sk" INTEGER, "c_salutation" CHAR(10),
+  "c_first_name" CHAR(20), "c_last_name" CHAR(30), "c_preferred_cust_flag" CHAR(1), "c_birth_day" INTEGER, "c_birth_month" INTEGER,
+  "c_birth_year" INTEGER, "c_birth_country" VARCHAR(20), "c_login" CHAR(13), "c_email_address" CHAR(50), "c_last_review_date" CHAR(10))' ),
+dboutput ('jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf;create=true','user1','passwd1',
+'CREATE TABLE CUSTOMER_ADDRESS ("ca_address_sk" INTEGER, "ca_address_id" CHAR(16), "ca_street_number" CHAR(10), "ca_street_name" VARCHAR(60),
+  "ca_street_type" CHAR(15), "ca_suite_number" CHAR(10), "ca_city" VARCHAR(60), "ca_county" VARCHAR(30), "ca_state" CHAR(2),
+  "ca_zip" CHAR(10), "ca_country" VARCHAR(20), "ca_gmt_offset" DECIMAL(5,2), "ca_location_type" CHAR(20))' ),
+dboutput ('jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf;create=true','user1','passwd1',
+'CREATE TABLE CUSTOMER_DEMOGRAPHICS ("cd_demo_sk" INTEGER, "cd_gender" CHAR(1), "cd_marital_status" CHAR(1), "cd_education_status" CHAR(20),
+  "cd_purchase_estimate" INTEGER, "cd_credit_rating" CHAR(10), "cd_dep_count" INTEGER, "cd_dep_employed_count" INTEGER,
+  "cd_dep_college_count" INTEGER)' ),
+dboutput ('jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf;create=true','user1','passwd1',
+'CREATE TABLE INVENTORY ("inv_date_sk" INTEGER, "inv_item_sk" INTEGER, "inv_warehouse_sk" INTEGER, "inv_quantity_on_hand" INTEGER)' ),
+dboutput ('jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf;create=true','user1','passwd1',
+'CREATE TABLE ITEM ("i_item_sk" INTEGER, "i_item_id" CHAR(16), "i_rec_start_date" DATE, "i_rec_end_date" DATE,
+  "i_item_desc" VARCHAR(200), "i_current_price" DECIMAL(7,2), "i_wholesale_cost" DECIMAL(7,2), "i_brand_id" INTEGER,
+  "i_brand" CHAR(50), "i_class_id" INTEGER, "i_class" CHAR(50), "i_category_id" INTEGER, "i_category" CHAR(50),
+  "i_manufact_id" INTEGER, "i_manufact" CHAR(50), "i_size" CHAR(20), "i_formulation" CHAR(20), "i_color" CHAR(20),
+  "i_units" CHAR(10), "i_container" CHAR(10), "i_manager_id" INTEGER, "i_product_name" CHAR(50))' ),
+dboutput ('jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf;create=true','user1','passwd1',
+'CREATE TABLE WAREHOUSE ("w_warehouse_sk" INTEGER, "w_warehouse_id" CHAR(16), "w_warehouse_name" VARCHAR(20),
+  "w_warehouse_sq_ft" INTEGER, "w_street_number" CHAR(10), "w_street_name" VARCHAR(60), "w_street_type" CHAR(15),
+  "w_suite_number" CHAR(10), "w_city" VARCHAR(60), "w_county" VARCHAR(30), "w_state" CHAR(2), "w_zip" CHAR(10),
+  "w_country" VARCHAR(20), "w_gmt_offset" DECIMAL(5,2))' ),
+dboutput ('jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf;create=true','user1','passwd1',
+'CREATE TABLE DATE_DIM ("d_date_sk" INTEGER, "d_date_id" CHAR(16), "d_date" DATE, "d_month_seq" INTEGER, "d_week_seq" INTEGER,
+  "d_quarter_seq" INTEGER, "d_year" INTEGER, "d_dow" INTEGER, "d_moy" INTEGER, "d_dom" INTEGER, "d_qoy" INTEGER,
+  "d_fy_year" INTEGER, "d_fy_quarter_seq" INTEGER, "d_fy_week_seq" INTEGER, "d_day_name" CHAR(9), "d_quarter_name" CHAR(6),
+  "d_holiday" CHAR(1), "d_weekend" CHAR(1), "d_following_holiday" CHAR(1), "d_first_dom" INTEGER, "d_last_dom" INTEGER,
+  "d_same_day_ly" INTEGER, "d_same_day_lq" INTEGER, "d_current_day" CHAR(1), "d_current_week" CHAR(1), "d_current_month" CHAR(1),
+  "d_current_quarter" CHAR(1), "d_current_year" CHAR(1))' ),
+dboutput ('jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf;create=true','user1','passwd1',
+'CREATE TABLE STORE ("s_store_sk" INTEGER, "s_store_id" CHAR(16), "s_rec_start_date" DATE, "s_rec_end_date" DATE,
+  "s_closed_date_sk" INTEGER, "s_store_name" VARCHAR(50), "s_number_employees" INTEGER, "s_floor_space" INTEGER,
+  "s_hours" CHAR(20), "s_manager" VARCHAR(40), "s_market_id" INTEGER, "s_geography_class" VARCHAR(100),
+  "s_market_desc" VARCHAR(100), "s_market_manager" VARCHAR(40), "s_division_id" INTEGER, "s_division_name" VARCHAR(50),
+  "s_company_id" INTEGER, "s_company_name" VARCHAR(50), "s_street_number" VARCHAR(10), "s_street_name" VARCHAR(60),
+  "s_street_type" CHAR(15), "s_suite_number" CHAR(10), "s_city" VARCHAR(60), "s_county" VARCHAR(30),
+  "s_state" CHAR(2), "s_zip" CHAR(10), "s_country" VARCHAR(20), "s_gmt_offset" DECIMAL(5,2), "s_tax_precentage" DECIMAL(5,2))' ),
+dboutput ('jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf;create=true','user1','passwd1',
+'CREATE TABLE HOUSEHOLD_DEMOGRAPHICS ("hd_demo_sk" INTEGER, "hd_income_band_sk" INTEGER, "hd_buy_potential" CHAR(15),
+  "hd_dep_count" INTEGER, "hd_vehicle_count" INTEGER)' ),
+dboutput ('jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf;create=true','user1','passwd1',
+'CREATE TABLE TIME_DIM ("t_time_sk" INTEGER, "t_time_id" CHAR(16), "t_time" INTEGER, "t_hour" INTEGER,
+  "t_minute" INTEGER, "t_second" INTEGER, "t_am_pm" CHAR(2), "t_shift" CHAR(20), "t_sub_shift" CHAR(20),
+  "t_meal_time" CHAR(20))' ),
+dboutput ('jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf;create=true','user1','passwd1',
+'CREATE TABLE REASON ("r_reason_sk" INTEGER, "r_reason_id" CHAR(16), "r_reason_desc" CHAR(100))' )
+limit 1;
+
+
+CREATE EXTERNAL TABLE catalog_sales
+(
+    cs_sold_date_sk           int                           ,
+    cs_sold_time_sk           int                           ,
+    cs_ship_date_sk           int                           ,
+    cs_bill_customer_sk       int                           ,
+    cs_bill_cdemo_sk          int                           ,
+    cs_bill_hdemo_sk          int                           ,
+    cs_bill_addr_sk           int                           ,
+    cs_ship_customer_sk       int                           ,
+    cs_ship_cdemo_sk          int                           ,
+    cs_ship_hdemo_sk          int                           ,
+    cs_ship_addr_sk           int                           ,
+    cs_call_center_sk         int                           ,
+    cs_catalog_page_sk        int                           ,
+    cs_ship_mode_sk           int                           ,
+    cs_warehouse_sk           int                           ,
+    cs_item_sk                int                           ,
+    cs_promo_sk               int                           ,
+    cs_order_number           int                           ,
+    cs_quantity               int                           ,
+    cs_wholesale_cost         decimal(7,2)                  ,
+    cs_list_price             decimal(7,2)                  ,
+    cs_sales_price            decimal(7,2)                  ,
+    cs_ext_discount_amt       decimal(7,2)                  ,
+    cs_ext_sales_price        decimal(7,2)                  ,
+    cs_ext_wholesale_cost     decimal(7,2)                  ,
+    cs_ext_list_price         decimal(7,2)                  ,
+    cs_ext_tax                decimal(7,2)                  ,
+    cs_coupon_amt             decimal(7,2)                  ,
+    cs_ext_ship_cost          decimal(7,2)                  ,
+    cs_net_paid               decimal(7,2)                  ,
+    cs_net_paid_inc_tax       decimal(7,2)                  ,
+    cs_net_paid_inc_ship      decimal(7,2)                  ,
+    cs_net_paid_inc_ship_tax  decimal(7,2)                  ,
+    cs_net_profit             decimal(7,2)
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+                "hive.sql.jdbc.url" = "jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf;collation=TERRITORY_BASED:PRIMARY",
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "CATALOG_SALES",
+                "hive.sql.dbcp.maxActive" = "1"
+);
+
+CREATE EXTERNAL TABLE catalog_returns
+(
+    cr_returned_date_sk       int                           ,
+    cr_returned_time_sk       int                           ,
+    cr_item_sk                int                           ,
+    cr_refunded_customer_sk   int                           ,
+    cr_refunded_cdemo_sk      int                           ,
+    cr_refunded_hdemo_sk      int                           ,
+    cr_refunded_addr_sk       int                           ,
+    cr_returning_customer_sk  int                           ,
+    cr_returning_cdemo_sk     int                           ,
+    cr_returning_hdemo_sk     int                           ,
+    cr_returning_addr_sk      int                           ,
+    cr_call_center_sk         int                           ,
+    cr_catalog_page_sk        int                           ,
+    cr_ship_mode_sk           int                           ,
+    cr_warehouse_sk           int                           ,
+    cr_reason_sk              int                           ,
+    cr_order_number           int                           ,
+    cr_return_quantity        int                           ,
+    cr_return_amount          decimal(7,2)                  ,
+    cr_return_tax             decimal(7,2)                  ,
+    cr_return_amt_inc_tax     decimal(7,2)                  ,
+    cr_fee                    decimal(7,2)                  ,
+    cr_return_ship_cost       decimal(7,2)                  ,
+    cr_refunded_cash          decimal(7,2)                  ,
+    cr_reversed_charge        decimal(7,2)                  ,
+    cr_store_credit           decimal(7,2)                  ,
+    cr_net_loss               decimal(7,2)
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+                "hive.sql.jdbc.url" = "jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf;collation=TERRITORY_BASED:PRIMARY",
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "CATALOG_RETURNS",
+                "hive.sql.dbcp.maxActive" = "1"
+);
+
+CREATE EXTERNAL TABLE store_sales
+(
+    ss_sold_date_sk           int                           ,
+    ss_sold_time_sk           int                           ,
+    ss_item_sk                int                           ,
+    ss_customer_sk            int                           ,
+    ss_cdemo_sk               int                           ,
+    ss_hdemo_sk               int                           ,
+    ss_addr_sk                int                           ,
+    ss_store_sk               int                           ,
+    ss_promo_sk               int                           ,
+    ss_ticket_number          int                           ,
+    ss_quantity               int                           ,
+    ss_wholesale_cost         decimal(7,2)                  ,
+    ss_list_price             decimal(7,2)                  ,
+    ss_sales_price            decimal(7,2)                  ,
+    ss_ext_discount_amt       decimal(7,2)                  ,
+    ss_ext_sales_price        decimal(7,2)                  ,
+    ss_ext_wholesale_cost     decimal(7,2)                  ,
+    ss_ext_list_price         decimal(7,2)                  ,
+    ss_ext_tax                decimal(7,2)                  ,
+    ss_coupon_amt             decimal(7,2)                  ,
+    ss_net_paid               decimal(7,2)                  ,
+    ss_net_paid_inc_tax       decimal(7,2)                  ,
+    ss_net_profit             decimal(7,2)
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+                "hive.sql.jdbc.url" = "jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf;collation=TERRITORY_BASED:PRIMARY",
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "STORE_SALES",
+                "hive.sql.dbcp.maxActive" = "1"
+);
+
+CREATE EXTERNAL TABLE store_returns
+(
+    sr_returned_date_sk       int                           ,
+    sr_return_time_sk         int                           ,
+    sr_item_sk                int                           ,
+    sr_customer_sk            int                           ,
+    sr_cdemo_sk               int                           ,
+    sr_hdemo_sk               int                           ,
+    sr_addr_sk                int                           ,
+    sr_store_sk               int                           ,
+    sr_reason_sk              int                           ,
+    sr_ticket_number          int                           ,
+    sr_return_quantity        int                           ,
+    sr_return_amt             decimal(7,2)                  ,
+    sr_return_tax             decimal(7,2)                  ,
+    sr_return_amt_inc_tax     decimal(7,2)                  ,
+    sr_fee                    decimal(7,2)                  ,
+    sr_return_ship_cost       decimal(7,2)                  ,
+    sr_refunded_cash          decimal(7,2)                  ,
+    sr_reversed_charge        decimal(7,2)                  ,
+    sr_store_credit           decimal(7,2)                  ,
+    sr_net_loss               decimal(7,2)
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+                "hive.sql.jdbc.url" = "jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf;collation=TERRITORY_BASED:PRIMARY",
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "STORE_RETURNS",
+                "hive.sql.dbcp.maxActive" = "1"
+);
+
+CREATE EXTERNAL TABLE web_sales
+(
+    ws_sold_date_sk           int                           ,
+    ws_sold_time_sk           int                           ,
+    ws_ship_date_sk           int                           ,
+    ws_item_sk                int                           ,
+    ws_bill_customer_sk       int                           ,
+    ws_bill_cdemo_sk          int                           ,
+    ws_bill_hdemo_sk          int                           ,
+    ws_bill_addr_sk           int                           ,
+    ws_ship_customer_sk       int                           ,
+    ws_ship_cdemo_sk          int                           ,
+    ws_ship_hdemo_sk          int                           ,
+    ws_ship_addr_sk           int                           ,
+    ws_web_page_sk            int                           ,
+    ws_web_site_sk            int                           ,
+    ws_ship_mode_sk           int                           ,
+    ws_warehouse_sk           int                           ,
+    ws_promo_sk               int                           ,
+    ws_order_number           int                           ,
+    ws_quantity               int                           ,
+    ws_wholesale_cost         decimal(7,2)                  ,
+    ws_list_price             decimal(7,2)                  ,
+    ws_sales_price            decimal(7,2)                  ,
+    ws_ext_discount_amt       decimal(7,2)                  ,
+    ws_ext_sales_price        decimal(7,2)                  ,
+    ws_ext_wholesale_cost     decimal(7,2)                  ,
+    ws_ext_list_price         decimal(7,2)                  ,
+    ws_ext_tax                decimal(7,2)                  ,
+    ws_coupon_amt             decimal(7,2)                  ,
+    ws_ext_ship_cost          decimal(7,2)                  ,
+    ws_net_paid               decimal(7,2)                  ,
+    ws_net_paid_inc_tax       decimal(7,2)                  ,
+    ws_net_paid_inc_ship      decimal(7,2)                  ,
+    ws_net_paid_inc_ship_tax  decimal(7,2)                  ,
+    ws_net_profit             decimal(7,2)
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+                "hive.sql.jdbc.url" = "jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf;collation=TERRITORY_BASED:PRIMARY",
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "WEB_SALES",
+                "hive.sql.dbcp.maxActive" = "1"
+);
+
+CREATE EXTERNAL TABLE web_returns
+(
+    wr_returned_date_sk       int                           ,
+    wr_returned_time_sk       int                           ,
+    wr_item_sk                int                           ,
+    wr_refunded_customer_sk   int                           ,
+    wr_refunded_cdemo_sk      int                           ,
+    wr_refunded_hdemo_sk      int                           ,
+    wr_refunded_addr_sk       int                           ,
+    wr_returning_customer_sk  int                           ,
+    wr_returning_cdemo_sk     int                           ,
+    wr_returning_hdemo_sk     int                           ,
+    wr_returning_addr_sk      int                           ,
+    wr_web_page_sk            int                           ,
+    wr_reason_sk              int                           ,
+    wr_order_number           int                           ,
+    wr_return_quantity        int                           ,
+    wr_return_amt             decimal(7,2)                  ,
+    wr_return_tax             decimal(7,2)                  ,
+    wr_return_amt_inc_tax     decimal(7,2)                  ,
+    wr_fee                    decimal(7,2)                  ,
+    wr_return_ship_cost       decimal(7,2)                  ,
+    wr_refunded_cash          decimal(7,2)                  ,
+    wr_reversed_charge        decimal(7,2)                  ,
+    wr_account_credit         decimal(7,2)                  ,
+    wr_net_loss               decimal(7,2)
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+                "hive.sql.jdbc.url" = "jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf;collation=TERRITORY_BASED:PRIMARY",
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "WEB_RETURNS",
+                "hive.sql.dbcp.maxActive" = "1"
+);
+
+CREATE EXTERNAL TABLE customer
+(
+    c_customer_sk             int                           ,
+    c_customer_id             char(16)                      ,
+    c_current_cdemo_sk        int                           ,
+    c_current_hdemo_sk        int                           ,
+    c_current_addr_sk         int                           ,
+    c_first_shipto_date_sk    int                           ,
+    c_first_sales_date_sk     int                           ,
+    c_salutation              char(10)                      ,
+    c_first_name              char(20)                      ,
+    c_last_name               char(30)                      ,
+    c_preferred_cust_flag     char(1)                       ,
+    c_birth_day               int                           ,
+    c_birth_month             int                           ,
+    c_birth_year              int                           ,
+    c_birth_country           varchar(20)                   ,
+    c_login                   char(13)                      ,
+    c_email_address           char(50)                      ,
+    c_last_review_date        char(10)
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+                "hive.sql.jdbc.url" = "jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf;collation=TERRITORY_BASED:PRIMARY",
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "CUSTOMER",
+                "hive.sql.dbcp.maxActive" = "1"
+);
+
+CREATE EXTERNAL TABLE customer_address
+(
+    ca_address_sk             int                           ,
+    ca_address_id             char(16)                      ,
+    ca_street_number          char(10)                      ,
+    ca_street_name            varchar(60)                   ,
+    ca_street_type            char(15)                      ,
+    ca_suite_number           char(10)                      ,
+    ca_city                   varchar(60)                   ,
+    ca_county                 varchar(30)                   ,
+    ca_state                  char(2)                       ,
+    ca_zip                    char(10)                      ,
+    ca_country                varchar(20)                   ,
+    ca_gmt_offset             decimal(5,2)                  ,
+    ca_location_type          char(20)
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+                "hive.sql.jdbc.url" = "jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf;collation=TERRITORY_BASED:PRIMARY",
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "CUSTOMER_ADDRESS",
+                "hive.sql.dbcp.maxActive" = "1"
+);
+
+CREATE EXTERNAL TABLE customer_demographics
+(
+    cd_demo_sk                int                           ,
+    cd_gender                 char(1)                       ,
+    cd_marital_status         char(1)                       ,
+    cd_education_status       char(20)                      ,
+    cd_purchase_estimate      int                           ,
+    cd_credit_rating          char(10)                      ,
+    cd_dep_count              int                           ,
+    cd_dep_employed_count     int                           ,
+    cd_dep_college_count      int
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+                "hive.sql.jdbc.url" = "jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf;collation=TERRITORY_BASED:PRIMARY",
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "CUSTOMER_DEMOGRAPHICS",
+                "hive.sql.dbcp.maxActive" = "1"
+);
+
+CREATE EXTERNAL TABLE inventory
+(
+    inv_date_sk               int                           ,
+    inv_item_sk               int                           ,
+    inv_warehouse_sk          int                           ,
+    inv_quantity_on_hand      int
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+                "hive.sql.jdbc.url" = "jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf;collation=TERRITORY_BASED:PRIMARY",
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "INVENTORY",
+                "hive.sql.dbcp.maxActive" = "1"
+);
+
+CREATE EXTERNAL TABLE item
+(
+    i_item_sk                 int                           ,
+    i_item_id                 char(16)                      ,
+    i_rec_start_date          date                          ,
+    i_rec_end_date            date                          ,
+    i_item_desc               varchar(200)                  ,
+    i_current_price           decimal(7,2)                  ,
+    i_wholesale_cost          decimal(7,2)                  ,
+    i_brand_id                int                           ,
+    i_brand                   char(50)                      ,
+    i_class_id                int                           ,
+    i_class                   char(50)                      ,
+    i_category_id             int                           ,
+    i_category                char(50)                      ,
+    i_manufact_id             int                           ,
+    i_manufact                char(50)                      ,
+    i_size                    char(20)                      ,
+    i_formulation             char(20)                      ,
+    i_color                   char(20)                      ,
+    i_units                   char(10)                      ,
+    i_container               char(10)                      ,
+    i_manager_id              int                           ,
+    i_product_name            char(50)
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+                "hive.sql.jdbc.url" = "jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf;collation=TERRITORY_BASED:PRIMARY",
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "ITEM",
+                "hive.sql.dbcp.maxActive" = "1"
+);
+
+CREATE EXTERNAL TABLE warehouse
+(
+    w_warehouse_sk            int                           ,
+    w_warehouse_id            char(16)                      ,
+    w_warehouse_name          varchar(20)                   ,
+    w_warehouse_sq_ft         int                           ,
+    w_street_number           char(10)                      ,
+    w_street_name             varchar(60)                   ,
+    w_street_type             char(15)                      ,
+    w_suite_number            char(10)                      ,
+    w_city                    varchar(60)                   ,
+    w_county                  varchar(30)                   ,
+    w_state                   char(2)                       ,
+    w_zip                     char(10)                      ,
+    w_country                 varchar(20)                   ,
+    w_gmt_offset              decimal(5,2)
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+                "hive.sql.jdbc.url" = "jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf;collation=TERRITORY_BASED:PRIMARY",
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "WAREHOUSE",
+                "hive.sql.dbcp.maxActive" = "1"
+);
+
+CREATE EXTERNAL TABLE date_dim
+(
+    d_date_sk                 int                           ,
+    d_date_id                 char(16)                      ,
+    d_date                    date                          ,
+    d_month_seq               int                           ,
+    d_week_seq                int                           ,
+    d_quarter_seq             int                           ,
+    d_year                    int                           ,
+    d_dow                     int                           ,
+    d_moy                     int                           ,
+    d_dom                     int                           ,
+    d_qoy                     int                           ,
+    d_fy_year                 int                           ,
+    d_fy_quarter_seq          int                           ,
+    d_fy_week_seq             int                           ,
+    d_day_name                char(9)                       ,
+    d_quarter_name            char(6)                       ,
+    d_holiday                 char(1)                       ,
+    d_weekend                 char(1)                       ,
+    d_following_holiday       char(1)                       ,
+    d_first_dom               int                           ,
+    d_last_dom                int                           ,
+    d_same_day_ly             int                           ,
+    d_same_day_lq             int                           ,
+    d_current_day             char(1)                       ,
+    d_current_week            char(1)                       ,
+    d_current_month           char(1)                       ,
+    d_current_quarter         char(1)                       ,
+    d_current_year            char(1)
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+                "hive.sql.jdbc.url" = "jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf;collation=TERRITORY_BASED:PRIMARY",
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "DATE_DIM",
+                "hive.sql.dbcp.maxActive" = "1"
+);
+
+CREATE EXTERNAL TABLE store
+(
+    s_store_sk                int                           ,
+    s_store_id                char(16)                      ,
+    s_rec_start_date          date                          ,
+    s_rec_end_date            date                          ,
+    s_closed_date_sk          int                           ,
+    s_store_name              varchar(50)                   ,
+    s_number_employees        int                           ,
+    s_floor_space             int                           ,
+    s_hours                   char(20)                      ,
+    s_manager                 varchar(40)                   ,
+    s_market_id               int                           ,
+    s_geography_class         varchar(100)                  ,
+    s_market_desc             varchar(100)                  ,
+    s_market_manager          varchar(40)                   ,
+    s_division_id             int                           ,
+    s_division_name           varchar(50)                   ,
+    s_company_id              int                           ,
+    s_company_name            varchar(50)                   ,
+    s_street_number           varchar(10)                   ,
+    s_street_name             varchar(60)                   ,
+    s_street_type             char(15)                      ,
+    s_suite_number            char(10)                      ,
+    s_city                    varchar(60)                   ,
+    s_county                  varchar(30)                   ,
+    s_state                   char(2)                       ,
+    s_zip                     char(10)                      ,
+    s_country                 varchar(20)                   ,
+    s_gmt_offset              decimal(5,2)                  ,
+    s_tax_precentage          decimal(5,2)                  
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+                "hive.sql.jdbc.url" = "jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf;collation=TERRITORY_BASED:PRIMARY",
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "STORE",
+                "hive.sql.dbcp.maxActive" = "1"
+);
+
+CREATE EXTERNAL TABLE household_demographics
+(
+    hd_demo_sk                int                           ,
+    hd_income_band_sk         int                           ,
+    hd_buy_potential          char(15)                      ,
+    hd_dep_count              int                           ,
+    hd_vehicle_count          int                           
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+                "hive.sql.jdbc.url" = "jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf;collation=TERRITORY_BASED:PRIMARY",
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "HOUSEHOLD_DEMOGRAPHICS",
+                "hive.sql.dbcp.maxActive" = "1"
+);
+
+CREATE EXTERNAL TABLE time_dim
+(
+    t_time_sk                 int                           ,
+    t_time_id                 char(16)                      ,
+    t_time                    int                           ,
+    t_hour                    int                           ,
+    t_minute                  int                           ,
+    t_second                  int                           ,
+    t_am_pm                   char(2)                       ,
+    t_shift                   char(20)                      ,
+    t_sub_shift               char(20)                      ,
+    t_meal_time               char(20)                      
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+                "hive.sql.jdbc.url" = "jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf;collation=TERRITORY_BASED:PRIMARY",
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "TIME_DIM",
+                "hive.sql.dbcp.maxActive" = "1"
+);
+
+CREATE EXTERNAL TABLE reason
+(
+    r_reason_sk               int                           ,
+    r_reason_id               char(16)                      ,
+    r_reason_desc             char(100)
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+                "hive.sql.jdbc.url" = "jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf;collation=TERRITORY_BASED:PRIMARY",
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "REASON",
+                "hive.sql.dbcp.maxActive" = "1"
+);
+
+
+explain
+select inv1.w_warehouse_sk, inv1.i_item_sk, inv1.d_moy, inv1.mean, inv1.cov,
+       inv2.w_warehouse_sk, inv2.i_item_sk, inv2.d_moy, inv2.mean, inv2.cov
+from (select w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy,
+             stdev, mean, case mean when 0.0
+                                    then null else stdev/mean end cov
+      from (select w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy,
+                   sum(inv_quantity_on_hand) as stdev,
+                   avg(inv_quantity_on_hand) as mean
+            from inventory
+            join item on inventory.inv_item_sk = item.i_item_sk
+            join warehouse on inventory.inv_warehouse_sk = warehouse.w_warehouse_sk
+            join date_dim on inventory.inv_date_sk = date_dim.d_date_sk
+            where d_year = 2001
+            group by w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy) foo
+      where case mean when 0.0
+                      then 0.0 else stdev/mean end > 1) inv1
+join (select w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy,
+             stdev, mean, case mean when 0.0
+                                    then null else stdev/mean end cov
+      from (select w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy,
+                   sum(inv_quantity_on_hand) as stdev,
+                   avg(inv_quantity_on_hand) as mean
+            from inventory
+            join item on inventory.inv_item_sk = item.i_item_sk
+            join warehouse on inventory.inv_warehouse_sk = warehouse.w_warehouse_sk
+            join date_dim on inventory.inv_date_sk = date_dim.d_date_sk
+            where d_year = 2001
+            group by w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy) foo
+      where case mean when 0.0
+                      then 0.0 else stdev/mean end > 1) inv2
+  on inv1.i_item_sk = inv2.i_item_sk
+     and inv1.w_warehouse_sk = inv2.w_warehouse_sk
+where inv1.d_moy = 1 and inv2.d_moy = 1+1
+order by inv1.w_warehouse_sk, inv1.i_item_sk, inv1.d_moy, inv1.mean, inv1.cov,
+         inv2.d_moy, inv2.mean, inv2.cov;
+select inv1.w_warehouse_sk, inv1.i_item_sk, inv1.d_moy, inv1.mean, inv1.cov,
+       inv2.w_warehouse_sk, inv2.i_item_sk, inv2.d_moy, inv2.mean, inv2.cov
+from (select w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy,
+             stdev, mean, case mean when 0.0
+                                    then null else stdev/mean end cov
+      from (select w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy,
+                   sum(inv_quantity_on_hand) as stdev,
+                   avg(inv_quantity_on_hand) as mean
+            from inventory
+            join item on inventory.inv_item_sk = item.i_item_sk
+            join warehouse on inventory.inv_warehouse_sk = warehouse.w_warehouse_sk
+            join date_dim on inventory.inv_date_sk = date_dim.d_date_sk
+            where d_year = 2001
+            group by w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy) foo
+      where case mean when 0.0
+                      then 0.0 else stdev/mean end > 1) inv1
+join (select w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy,
+             stdev, mean, case mean when 0.0
+                                    then null else stdev/mean end cov
+      from (select w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy,
+                   sum(inv_quantity_on_hand) as stdev,
+                   avg(inv_quantity_on_hand) as mean
+            from inventory
+            join item on inventory.inv_item_sk = item.i_item_sk
+            join warehouse on inventory.inv_warehouse_sk = warehouse.w_warehouse_sk
+            join date_dim on inventory.inv_date_sk = date_dim.d_date_sk
+            where d_year = 2001
+            group by w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy) foo
+      where case mean when 0.0
+                      then 0.0 else stdev/mean end > 1) inv2
+  on inv1.i_item_sk = inv2.i_item_sk
+     and inv1.w_warehouse_sk = inv2.w_warehouse_sk
+where inv1.d_moy = 1 and inv2.d_moy = 1+1
+order by inv1.w_warehouse_sk, inv1.i_item_sk, inv1.d_moy, inv1.mean, inv1.cov,
+         inv2.d_moy, inv2.mean, inv2.cov;
+
+
+explain
+SELECT cd_gender, 
+       cd_marital_status, 
+       cd_education_status, 
+       Count(*) cnt1, 
+       cd_purchase_estimate, 
+       Count(*) cnt2, 
+       cd_credit_rating, 
+       Count(*) cnt3 
+FROM   customer c, 
+       customer_address ca, 
+       customer_demographics 
+WHERE  c.c_current_addr_sk = ca.ca_address_sk 
+       AND ca_state IN ( 'CO', 'IL', 'MN' ) 
+       AND cd_demo_sk = c.c_current_cdemo_sk 
+       AND EXISTS (SELECT * 
+                   FROM   store_sales, 
+                          date_dim 
+                   WHERE  c.c_customer_sk = ss_customer_sk 
+                          AND ss_sold_date_sk = d_date_sk 
+                          AND d_year = 1999 
+                          AND d_moy BETWEEN 1 AND 1 + 2) 
+       AND ( NOT EXISTS (SELECT * 
+                         FROM   web_sales, 
+                                date_dim 
+                         WHERE  c.c_customer_sk = ws_bill_customer_sk 
+                                AND ws_sold_date_sk = d_date_sk 
+                                AND d_year = 1999 
+                                AND d_moy BETWEEN 1 AND 1 + 2) 
+             AND NOT EXISTS (SELECT * 
+                             FROM   catalog_sales, 
+                                    date_dim 
+                             WHERE  c.c_customer_sk = cs_ship_customer_sk 
+                                    AND cs_sold_date_sk = d_date_sk 
+                                    AND d_year = 1999 
+                                    AND d_moy BETWEEN 1 AND 1 + 2) ) 
+GROUP  BY cd_gender, 
+          cd_marital_status, 
+          cd_education_status, 
+          cd_purchase_estimate, 
+          cd_credit_rating 
+ORDER  BY cd_gender, 
+          cd_marital_status, 
+          cd_education_status, 
+          cd_purchase_estimate, 
+          cd_credit_rating 
+LIMIT  100; 
+SELECT cd_gender, 
+       cd_marital_status, 
+       cd_education_status, 
+       Count(*) cnt1, 
+       cd_purchase_estimate, 
+       Count(*) cnt2, 
+       cd_credit_rating, 
+       Count(*) cnt3 
+FROM   customer c, 
+       customer_address ca, 
+       customer_demographics 
+WHERE  c.c_current_addr_sk = ca.ca_address_sk 
+       AND ca_state IN ( 'CO', 'IL', 'MN' ) 
+       AND cd_demo_sk = c.c_current_cdemo_sk 
+       AND EXISTS (SELECT * 
+                   FROM   store_sales, 
+                          date_dim 
+                   WHERE  c.c_customer_sk = ss_customer_sk 
+                          AND ss_sold_date_sk = d_date_sk 
+                          AND d_year = 1999 
+                          AND d_moy BETWEEN 1 AND 1 + 2) 
+       AND ( NOT EXISTS (SELECT * 
+                         FROM   web_sales, 
+                                date_dim 
+                         WHERE  c.c_customer_sk = ws_bill_customer_sk 
+                                AND ws_sold_date_sk = d_date_sk 
+                                AND d_year = 1999 
+                                AND d_moy BETWEEN 1 AND 1 + 2) 
+             AND NOT EXISTS (SELECT * 
+                             FROM   catalog_sales, 
+                                    date_dim 
+                             WHERE  c.c_customer_sk = cs_ship_customer_sk 
+                                    AND cs_sold_date_sk = d_date_sk 
+                                    AND d_year = 1999 
+                                    AND d_moy BETWEEN 1 AND 1 + 2) ) 
+GROUP  BY cd_gender, 
+          cd_marital_status, 
+          cd_education_status, 
+          cd_purchase_estimate, 
+          cd_credit_rating 
+ORDER  BY cd_gender, 
+          cd_marital_status, 
+          cd_education_status, 
+          cd_purchase_estimate, 
+          cd_credit_rating 
+LIMIT  100;
+
+
+explain
+SELECT cd_gender, 
+       cd_marital_status, 
+       cd_education_status, 
+       Count(*) cnt1, 
+       cd_purchase_estimate, 
+       Count(*) cnt2, 
+       cd_credit_rating, 
+       Count(*) cnt3 
+FROM   customer c, 
+       customer_address ca, 
+       customer_demographics 
+WHERE  c.c_current_addr_sk = ca.ca_address_sk 
+       AND ca_state IN ( 'CO', 'IL', 'MN' ) 
+       AND cd_demo_sk = c.c_current_cdemo_sk 
+       AND EXISTS (SELECT * 
+                   FROM   store_sales, 
+                          date_dim 
+                   WHERE  c.c_customer_sk = ss_customer_sk 
+                          AND ss_sold_date_sk = d_date_sk 
+                          AND d_year = 1999 
+                          AND d_moy NOT BETWEEN 1 AND 1 + 2) 
+       AND ( NOT EXISTS (SELECT * 
+                         FROM   web_sales, 
+                                date_dim 
+                         WHERE  c.c_customer_sk = ws_bill_customer_sk 
+                                AND ws_sold_date_sk = d_date_sk 
+                                AND d_year = 1999 
+                                AND d_moy NOT BETWEEN 1 AND 1 + 2) 
+             AND NOT EXISTS (SELECT * 
+                             FROM   catalog_sales, 
+                                    date_dim 
+                             WHERE  c.c_customer_sk = cs_ship_customer_sk 
+                                    AND cs_sold_date_sk = d_date_sk 
+                                    AND d_year = 1999 
+                                    AND d_moy NOT BETWEEN 1 AND 1 + 2) ) 
+GROUP  BY cd_gender, 
+          cd_marital_status, 
+          cd_education_status, 
+          cd_purchase_estimate, 
+          cd_credit_rating 
+ORDER  BY cd_gender, 
+          cd_marital_status, 
+          cd_education_status, 
+          cd_purchase_estimate, 
+          cd_credit_rating 
+LIMIT  100; 
+SELECT cd_gender, 
+       cd_marital_status, 
+       cd_education_status, 
+       Count(*) cnt1, 
+       cd_purchase_estimate, 
+       Count(*) cnt2, 
+       cd_credit_rating, 
+       Count(*) cnt3 
+FROM   customer c, 
+       customer_address ca, 
+       customer_demographics 
+WHERE  c.c_current_addr_sk = ca.ca_address_sk 
+       AND ca_state IN ( 'CO', 'IL', 'MN' ) 
+       AND cd_demo_sk = c.c_current_cdemo_sk 
+       AND EXISTS (SELECT * 
+                   FROM   store_sales, 
+                          date_dim 
+                   WHERE  c.c_customer_sk = ss_customer_sk 
+                          AND ss_sold_date_sk = d_date_sk 
+                          AND d_year = 1999 
+                          AND d_moy NOT BETWEEN 1 AND 1 + 2) 
+       AND ( NOT EXISTS (SELECT * 
+                         FROM   web_sales, 
+                                date_dim 
+                         WHERE  c.c_customer_sk = ws_bill_customer_sk 
+                                AND ws_sold_date_sk = d_date_sk 
+                                AND d_year = 1999 
+                                AND d_moy NOT BETWEEN 1 AND 1 + 2) 
+             AND NOT EXISTS (SELECT * 
+                             FROM   catalog_sales, 
+                                    date_dim 
+                             WHERE  c.c_customer_sk = cs_ship_customer_sk 
+                                    AND cs_sold_date_sk = d_date_sk 
+                                    AND d_year = 1999 
+                                    AND d_moy NOT BETWEEN 1 AND 1 + 2) ) 
+GROUP  BY cd_gender, 
+          cd_marital_status, 
+          cd_education_status, 
+          cd_purchase_estimate, 
+          cd_credit_rating 
+ORDER  BY cd_gender, 
+          cd_marital_status, 
+          cd_education_status, 
+          cd_purchase_estimate, 
+          cd_credit_rating 
+LIMIT  100;
+
+
+explain
+SELECT Count(*) 
+FROM   (SELECT DISTINCT c_last_name, 
+                        c_first_name, 
+                        d_date 
+        FROM   store_sales, 
+               date_dim, 
+               customer 
+        WHERE  store_sales.ss_sold_date_sk = date_dim.d_date_sk 
+               AND store_sales.ss_customer_sk = customer.c_customer_sk 
+               AND d_month_seq BETWEEN 1212 AND 1212 + 11 
+        intersect 
+        SELECT DISTINCT c_last_name, 
+                        c_first_name, 
+                        d_date 
+        FROM   catalog_sales, 
+               date_dim, 
+               customer 
+        WHERE  catalog_sales.cs_sold_date_sk = date_dim.d_date_sk 
+               AND catalog_sales.cs_bill_customer_sk = customer.c_customer_sk 
+               AND d_month_seq BETWEEN 1212 AND 1212 + 11 
+        intersect 
+        SELECT DISTINCT c_last_name, 
+                        c_first_name, 
+                        d_date 
+        FROM   web_sales, 
+               date_dim, 
+               customer 
+        WHERE  web_sales.ws_sold_date_sk = date_dim.d_date_sk 
+               AND web_sales.ws_bill_customer_sk = customer.c_customer_sk 
+               AND d_month_seq BETWEEN 1212 AND 1212 + 11) hot_cust 
+LIMIT  100;
+SELECT Count(*) 
+FROM   (SELECT DISTINCT c_last_name, 
+                        c_first_name, 
+                        d_date 
+        FROM   store_sales, 
+               date_dim, 
+               customer 
+        WHERE  store_sales.ss_sold_date_sk = date_dim.d_date_sk 
+               AND store_sales.ss_customer_sk = customer.c_customer_sk 
+               AND d_month_seq BETWEEN 1212 AND 1212 + 11 
+        intersect 
+        SELECT DISTINCT c_last_name, 
+                        c_first_name, 
+                        d_date 
+        FROM   catalog_sales, 
+               date_dim, 
+               customer 
+        WHERE  catalog_sales.cs_sold_date_sk = date_dim.d_date_sk 
+               AND catalog_sales.cs_bill_customer_sk = customer.c_customer_sk 
+               AND d_month_seq BETWEEN 1212 AND 1212 + 11 
+        intersect 
+        SELECT DISTINCT c_last_name, 
+                        c_first_name, 
+                        d_date 
+        FROM   web_sales, 
+               date_dim, 
+               customer 
+        WHERE  web_sales.ws_sold_date_sk = date_dim.d_date_sk 
+               AND web_sales.ws_bill_customer_sk = customer.c_customer_sk 
+               AND d_month_seq BETWEEN 1212 AND 1212 + 11) hot_cust 
+LIMIT  100;
+
+
+explain
+WITH ss AS 
+( 
+         SELECT   i_item_id, 
+                  Sum(ss_ext_sales_price) total_sales 
+         FROM     store_sales, 
+                  date_dim, 
+                  customer_address, 
+                  item 
+         WHERE    i_item_id IN 
+                  ( 
+                         SELECT i_item_id 
+                         FROM   item 
+                         WHERE  i_color IN ('orchid', 
+                                            'chiffon', 
+                                            'lace')) 
+         AND      ss_item_sk = i_item_sk 
+         AND      ss_sold_date_sk = d_date_sk 
+         AND      d_year = 2000 
+         AND      d_moy = 1 
+         AND      ss_addr_sk = ca_address_sk 
+         AND      ca_gmt_offset = -8 
+         GROUP BY i_item_id), cs AS 
+( 
+         SELECT   i_item_id, 
+                  Sum(cs_ext_sales_price) total_sales 
+         FROM     catalog_sales, 
+                  date_dim, 
+                  customer_address, 
+                  item 
+         WHERE    i_item_id IN 
+                  ( 
+                         SELECT i_item_id 
+                         FROM   item 
+                         WHERE  i_color IN ('orchid', 
+                                            'chiffon', 
+                                            'lace')) 
+         AND      cs_item_sk = i_item_sk 
+         AND      cs_sold_date_sk = d_date_sk 
+         AND      d_year = 2000 
+         AND      d_moy = 1 
+         AND      cs_bill_addr_sk = ca_address_sk 
+         AND      ca_gmt_offset = -8 
+         GROUP BY i_item_id), ws AS 
+( 
+         SELECT   i_item_id, 
+                  Sum(ws_ext_sales_price) total_sales 
+         FROM     web_sales, 
+                  date_dim, 
+                  customer_address, 
+                  item 
+         WHERE    i_item_id IN 
+                  ( 
+                         SELECT i_item_id 
+                         FROM   item 
+                         WHERE  i_color IN ('orchid', 
+                                            'chiffon', 
+                                            'lace')) 
+         AND      ws_item_sk = i_item_sk 
+         AND      ws_sold_date_sk = d_date_sk 
+         AND      d_year = 2000 
+         AND      d_moy = 1 
+         AND      ws_bill_addr_sk = ca_address_sk 
+         AND      ca_gmt_offset = -8 
+         GROUP BY i_item_id) 
+SELECT   i_item_id , 
+         Sum(total_sales) total_sales 
+FROM     ( 
+                SELECT * 
+                FROM   ss 
+                UNION ALL 
+                SELECT * 
+                FROM   cs 
+                UNION ALL 
+                SELECT * 
+                FROM   ws) tmp1 
+GROUP BY i_item_id 
+ORDER BY total_sales limit 100;
+WITH ss AS 
+( 
+         SELECT   i_item_id, 
+                  Sum(ss_ext_sales_price) total_sales 
+         FROM     store_sales, 
+                  date_dim, 
+                  customer_address, 
+                  item 
+         WHERE    i_item_id IN 
+                  ( 
+                         SELECT i_item_id 
+                         FROM   item 
+                         WHERE  i_color IN ('orchid', 
+                                            'chiffon', 
+                                            'lace')) 
+         AND      ss_item_sk = i_item_sk 
+         AND      ss_sold_date_sk = d_date_sk 
+         AND      d_year = 2000 
+         AND      d_moy = 1 
+         AND      ss_addr_sk = ca_address_sk 
+         AND      ca_gmt_offset = -8 
+         GROUP BY i_item_id), cs AS 
+( 
+         SELECT   i_item_id, 
+                  Sum(cs_ext_sales_price) total_sales 
+         FROM     catalog_sales, 
+                  date_dim, 
+                  customer_address, 
+                  item 
+         WHERE    i_item_id IN 
+                  ( 
+                         SELECT i_item_id 
+                         FROM   item 
+                         WHERE  i_color IN ('orchid', 
+                                            'chiffon', 
+                                            'lace')) 
+         AND      cs_item_sk = i_item_sk 
+         AND      cs_sold_date_sk = d_date_sk 
+         AND      d_year = 2000 
+         AND      d_moy = 1 
+         AND      cs_bill_addr_sk = ca_address_sk 
+         AND      ca_gmt_offset = -8 
+         GROUP BY i_item_id), ws AS 
+( 
+         SELECT   i_item_id, 
+                  Sum(ws_ext_sales_price) total_sales 
+         FROM     web_sales, 
+                  date_dim, 
+                  customer_address, 
+                  item 
+         WHERE    i_item_id IN 
+                  ( 
+                         SELECT i_item_id 
+                         FROM   item 
+                         WHERE  i_color IN ('orchid', 
+                                            'chiffon', 
+                                            'lace')) 
+         AND      ws_item_sk = i_item_sk 
+         AND      ws_sold_date_sk = d_date_sk 
+         AND      d_year = 2000 
+         AND      d_moy = 1 
+         AND      ws_bill_addr_sk = ca_address_sk 
+         AND      ca_gmt_offset = -8 
+         GROUP BY i_item_id) 
+SELECT   i_item_id , 
+         Sum(total_sales) total_sales 
+FROM     ( 
+                SELECT * 
+                FROM   ss 
+                UNION ALL 
+                SELECT * 
+                FROM   cs 
+                UNION ALL 
+                SELECT * 
+                FROM   ws) tmp1 
+GROUP BY i_item_id 
+ORDER BY total_sales limit 100;
+
+
+explain
+WITH sr_items AS 
+( 
+         SELECT   i_item_id               item_id, 
+                  Sum(sr_return_quantity) sr_item_qty 
+         FROM     store_returns, 
+                  item, 
+                  date_dim 
+         WHERE    sr_item_sk = i_item_sk 
+         AND      d_date IN 
+                  ( 
+                         SELECT d_date 
+                         FROM   date_dim 
+                         WHERE  d_week_seq IN 
+                                ( 
+                                       SELECT d_week_seq 
+                                       FROM   date_dim 
+                                       WHERE  d_date IN ('1998-01-02', 
+                                                         '1998-10-15', 
+                                                         '1998-11-10'))) 
+         AND      sr_returned_date_sk = d_date_sk 
+         GROUP BY i_item_id), cr_items AS 
+( 
+         SELECT   i_item_id               item_id, 
+                  Sum(cr_return_quantity) cr_item_qty 
+         FROM     catalog_returns, 
+                  item, 
+                  date_dim 
+         WHERE    cr_item_sk = i_item_sk 
+         AND      d_date IN 
+                  ( 
+                         SELECT d_date 
+                         FROM   date_dim 
+                         WHERE  d_week_seq IN 
+                                ( 
+                                       SELECT d_week_seq 
+                                       FROM   date_dim 
+                                       WHERE  d_date IN ('1998-01-02', 
+                                                         '1998-10-15', 
+                                                         '1998-11-10'))) 
+         AND      cr_returned_date_sk = d_date_sk 
+         GROUP BY i_item_id), wr_items AS 
+( 
+         SELECT   i_item_id               item_id, 
+                  Sum(wr_return_quantity) wr_item_qty 
+         FROM     web_returns, 
+                  item, 
+                  date_dim 
+         WHERE    wr_item_sk = i_item_sk 
+         AND      d_date IN 
+                  ( 
+                         SELECT d_date 
+                         FROM   date_dim 
+                         WHERE  d_week_seq IN 
+                                ( 
+                                       SELECT d_week_seq 
+                                       FROM   date_dim 
+                                       WHERE  d_date IN ('1998-01-02', 
+                                                         '1998-10-15', 
+                                                         '1998-11-10'))) 
+         AND      wr_returned_date_sk = d_date_sk 
+         GROUP BY i_item_id) 
+SELECT   sr_items.item_id , 
+         sr_item_qty , 
+         sr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 sr_dev , 
+         cr_item_qty , 
+         cr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 cr_dev , 
+         wr_item_qty , 
+         wr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 wr_dev , 
+         (sr_item_qty+cr_item_qty+wr_item_qty)/3.0                   average 
+FROM     sr_items , 
+         cr_items , 
+         wr_items 
+WHERE    sr_items.item_id=cr_items.item_id 
+AND      sr_items.item_id=wr_items.item_id 
+ORDER BY sr_items.item_id , 
+         sr_item_qty limit 100;
+
+
+explain
+with ss as (
+			 select
+				  i_manufact_id,sum(ss_ext_sales_price) total_sales
+			 from
+				store_sales,
+				date_dim,
+				 customer_address,
+				 item
+			 where
+				 i_manufact_id in (select
+			  i_manufact_id
+			from
+			 item
+			where i_category in ('Books'))
+			 and     ss_item_sk              = i_item_sk
+			 and     ss_sold_date_sk         = d_date_sk
+			 and     d_year                  = 1999
+			 and     d_moy                   = 3
+			 and     ss_addr_sk              = ca_address_sk
+			 and     ca_gmt_offset           = -6 
+			 group by i_manufact_id),
+			 cs as (
+			 select
+				  i_manufact_id,sum(cs_ext_sales_price) total_sales
+			 from
+				catalog_sales,
+				date_dim,
+				 customer_address,
+				 item
+			 where
+				 i_manufact_id               in (select
+			  i_manufact_id
+			from
+			 item
+			where i_category in ('Books'))
+			 and     cs_item_sk              = i_item_sk
+			 and     cs_sold_date_sk         = d_date_sk
+			 and     d_year                  = 1999
+			 and     d_moy                   = 3
+			 and     cs_bill_addr_sk         = ca_address_sk
+			 and     ca_gmt_offset           = -6 
+			 group by i_manufact_id),
+			 ws as (
+			 select
+				  i_manufact_id,sum(ws_ext_sales_price) total_sales
+			 from
+				web_sales,
+				date_dim,
+				 customer_address,
+				 item
+			 where
+				 i_manufact_id               in (select
+			  i_manufact_id
+			from
+			 item
+			where i_category in ('Books'))
+			 and     ws_item_sk              = i_item_sk
+			 and     ws_sold_date_sk         = d_date_sk
+			 and     d_year                  = 1999
+			 and     d_moy                   = 3
+			 and     ws_bill_addr_sk         = ca_address_sk
+			 and     ca_gmt_offset           = -6
+			 group by i_manufact_id)
+			  select  i_manufact_id ,sum(total_sales) total_sales
+			 from  (select * from ss 
+				union all
+				select * from cs 
+				union all
+				select * from ws) tmp1
+			 group by i_manufact_id
+			 order by total_sales
+			limit 100;
+with ss as (
+			 select
+				  i_manufact_id,sum(ss_ext_sales_price) total_sales
+			 from
+				store_sales,
+				date_dim,
+				 customer_address,
+				 item
+			 where
+				 i_manufact_id in (select
+			  i_manufact_id
+			from
+			 item
+			where i_category in ('Books'))
+			 and     ss_item_sk              = i_item_sk
+			 and     ss_sold_date_sk         = d_date_sk
+			 and     d_year                  = 1999
+			 and     d_moy                   = 3
+			 and     ss_addr_sk              = ca_address_sk
+			 and     ca_gmt_offset           = -6 
+			 group by i_manufact_id),
+			 cs as (
+			 select
+				  i_manufact_id,sum(cs_ext_sales_price) total_sales
+			 from
+				catalog_sales,
+				date_dim,
+				 customer_address,
+				 item
+			 where
+				 i_manufact_id               in (select
+			  i_manufact_id
+			from
+			 item
+			where i_category in ('Books'))
+			 and     cs_item_sk              = i_item_sk
+			 and     cs_sold_date_sk         = d_date_sk
+			 and     d_year                  = 1999
+			 and     d_moy                   = 3
+			 and     cs_bill_addr_sk         = ca_address_sk
+			 and     ca_gmt_offset           = -6 
+			 group by i_manufact_id),
+			 ws as (
+			 select
+				  i_manufact_id,sum(ws_ext_sales_price) total_sales
+			 from
+				web_sales,
+				date_dim,
+				 customer_address,
+				 item
+			 where
+				 i_manufact_id               in (select
+			  i_manufact_id
+			from
+			 item
+			where i_category in ('Books'))
+			 and     ws_item_sk              = i_item_sk
+			 and     ws_sold_date_sk         = d_date_sk
+			 and     d_year                  = 1999
+			 and     d_moy                   = 3
+			 and     ws_bill_addr_sk         = ca_address_sk
+			 and     ca_gmt_offset           = -6
+			 group by i_manufact_id)
+			  select  i_manufact_id ,sum(total_sales) total_sales
+			 from  (select * from ss 
+				union all
+				select * from cs 
+				union all
+				select * from ws) tmp1
+			 group by i_manufact_id
+			 order by total_sales
+			limit 100;
+
+
+explain
+select *
+                          from (select count(*) as h8_30_to_9
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 8
+                                  and time_dim.t_minute >= 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s1 join
+                               (select count(*) as h9_to_9_30
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 9
+                                  and time_dim.t_minute < 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s2 join
+                               (select count(*) as h9_30_to_10
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 9
+                                  and time_dim.t_minute >= 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s3 join
+                               (select count(*) as h10_to_10_30
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 10
+                                  and time_dim.t_minute < 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s4 join
+                               (select count(*) as h10_30_to_11
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 10
+                                  and time_dim.t_minute >= 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s5 join
+                               (select count(*) as h11_to_11_30
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 11
+                                  and time_dim.t_minute < 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s6 join
+                               (select count(*) as h11_30_to_12
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 11
+                                  and time_dim.t_minute >= 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s7 join
+                               (select count(*) as h12_to_12_30
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 12
+                                  and time_dim.t_minute < 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s8;
+select *
+                          from (select count(*) as h8_30_to_9
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 8
+                                  and time_dim.t_minute >= 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s1 join
+                               (select count(*) as h9_to_9_30
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 9
+                                  and time_dim.t_minute < 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s2 join
+                               (select count(*) as h9_30_to_10
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 9
+                                  and time_dim.t_minute >= 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s3 join
+                               (select count(*) as h10_to_10_30
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 10
+                                  and time_dim.t_minute < 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s4 join
+                               (select count(*) as h10_30_to_11
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 10
+                                  and time_dim.t_minute >= 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s5 join
+                               (select count(*) as h11_to_11_30
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 11
+                                  and time_dim.t_minute < 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s6 join
+                               (select count(*) as h11_30_to_12
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 11
+                                  and time_dim.t_minute >= 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s7 join
+                               (select count(*) as h12_to_12_30
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 12
+                                  and time_dim.t_minute < 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s8;
+
+
+explain
+select substr(r_reason_desc, 1, 20) as r,
+avg(ws_quantity) as wq,
+avg(wr_refunded_cash) as ref,
+avg(wr_fee) as fee
+from web_sales
+join web_returns
+on web_sales.ws_item_sk = web_returns.wr_item_sk
+and web_sales.ws_order_number = web_returns.wr_order_number
+join customer_demographics cd1
+on cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk
+join customer_demographics cd2
+on cd2.cd_demo_sk = web_returns.wr_returning_cdemo_sk
+join customer_address
+on customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk
+join date_dim
+on web_sales.ws_sold_date_sk = date_dim.d_date_sk
+join reason
+on reason.r_reason_sk = web_returns.wr_reason_sk
+where d_year = 2000
+and ((cd1.cd_marital_status = 'M'
+and cd1.cd_marital_status = cd2.cd_marital_status
+and cd1.cd_education_status = 'Advanced Degree'
+and cd1.cd_education_status = cd2.cd_education_status
+and ws_sales_price between 100.00 and 150.00)
+or
+(cd1.cd_marital_status = 'S'
+and cd1.cd_marital_status = cd2.cd_marital_status
+and cd1.cd_education_status = 'College'
+and cd1.cd_education_status = cd2.cd_education_status
+and ws_sales_price between 50.00 and 100.00)
+or
+(cd1.cd_marital_status = 'W'
+and cd1.cd_marital_status = cd2.cd_marital_status
+and cd1.cd_education_status = '2 yr Degree'
+and cd1.cd_education_status = cd2.cd_education_status
+and ws_sales_price between 150.00 and 200.00))
+and ((ca_country = 'United States'
+and ca_state in ('IN', 'OH', 'NJ')
+and ws_net_profit between 100 and 200)
+or
+(ca_country = 'United States'
+and ca_state in ('WI', 'CT', 'KY')
+and ws_net_profit between 150 and 300)
+or
+(ca_country = 'United States'
+and ca_state in ('LA', 'IA', 'AR')
+and ws_net_profit between 50 and 250))
+group by r_reason_desc
+order by r, wq, ref, fee
+limit 100;
+select substr(r_reason_desc, 1, 20) as r,
+avg(ws_quantity) as wq,
+avg(wr_refunded_cash) as ref,
+avg(wr_fee) as fee
+from web_sales
+join web_returns
+on web_sales.ws_item_sk = web_returns.wr_item_sk
+and web_sales.ws_order_number = web_returns.wr_order_number
+join customer_demographics cd1
+on cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk
+join customer_demographics cd2
+on cd2.cd_demo_sk = web_returns.wr_returning_cdemo_sk
+join customer_address
+on customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk
+join date_dim
+on web_sales.ws_sold_date_sk = date_dim.d_date_sk
+join reason
+on reason.r_reason_sk = web_returns.wr_reason_sk
+where d_year = 2000
+and ((cd1.cd_marital_status = 'M'
+and cd1.cd_marital_status = cd2.cd_marital_status
+and cd1.cd_education_status = 'Advanced Degree'
+and cd1.cd_education_status = cd2.cd_education_status
+and ws_sales_price between 100.00 and 150.00)
+or
+(cd1.cd_marital_status = 'S'
+and cd1.cd_marital_status = cd2.cd_marital_status
+and cd1.cd_education_status = 'College'
+and cd1.cd_education_status = cd2.cd_education_status
+and ws_sales_price between 50.00 and 100.00)
+or
+(cd1.cd_marital_status = 'W'
+and cd1.cd_marital_status = cd2.cd_marital_status
+and cd1.cd_education_status = '2 yr Degree'
+and cd1.cd_education_status = cd2.cd_education_status
+and ws_sales_price between 150.00 and 200.00))
+and ((ca_country = 'United States'
+and ca_state in ('IN', 'OH', 'NJ')
+and ws_net_profit between 100 and 200)
+or
+(ca_country = 'United States'
+and ca_state in ('WI', 'CT', 'KY')
+and ws_net_profit between 150 and 300)
+or
+(ca_country = 'United States'
+and ca_state in ('LA', 'IA', 'AR')
+and ws_net_profit between 50 and 250))
+group by r_reason_desc
+order by r, wq, ref, fee
+limit 100;
+
+
+DROP TABLE catalog_sales;
+DROP TABLE catalog_returns;
+DROP TABLE store_sales;
+DROP TABLE store_returns;
+DROP TABLE web_sales;
+DROP TABLE web_returns;
+DROP TABLE customer;
+DROP TABLE customer_address;
+DROP TABLE customer_demographics;
+DROP TABLE inventory;
+DROP TABLE item;
+DROP TABLE warehouse;
+DROP TABLE date_dim;
+DROP TABLE store;
+DROP TABLE household_demographics;
+DROP TABLE time_dim;
+DROP TABLE reason;
+
+FROM src
+SELECT
+dboutput('jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf','user1','passwd1',
+'DROP TABLE CATALOG_SALES' ),
+dboutput('jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf','user1','passwd1',
+'DROP TABLE CATALOG_RETURNS' ),
+dboutput('jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf','user1','passwd1',
+'DROP TABLE STORE_SALES' ),
+dboutput('jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf','user1','passwd1',
+'DROP TABLE STORE_RETURNS' ),
+dboutput('jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf','user1','passwd1',
+'DROP TABLE WEB_SALES' ),
+dboutput('jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf','user1','passwd1',
+'DROP TABLE WEB_RETURNS' ),
+dboutput('jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf','user1','passwd1',
+'DROP TABLE CUSTOMER' ),
+dboutput('jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf','user1','passwd1',
+'DROP TABLE CUSTOMER_ADDRESS' ),
+dboutput('jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf','user1','passwd1',
+'DROP TABLE CUSTOMER_DEMOGRAPHICS' ),
+dboutput('jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf','user1','passwd1',
+'DROP TABLE INVENTORY' ),
+dboutput('jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf','user1','passwd1',
+'DROP TABLE ITEM' ),
+dboutput('jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf','user1','passwd1',
+'DROP TABLE WAREHOUSE' ),
+dboutput('jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf','user1','passwd1',
+'DROP TABLE DATE_DIM' ),
+dboutput('jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf','user1','passwd1',
+'DROP TABLE STORE' ),
+dboutput('jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf','user1','passwd1',
+'DROP TABLE HOUSEHOLD_DEMOGRAPHICS' ),
+dboutput('jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf','user1','passwd1',
+'DROP TABLE TIME_DIM' ),
+dboutput('jdbc:derby:;databaseName=${system:test.tmp.dir}/test_derby_perf','user1','passwd1',
+'DROP TABLE REASON' )
+limit 1;
diff --git a/ql/src/test/results/clientpositive/join34.q.out b/ql/src/test/results/clientpositive/join34.q.out
index b38a424..40c22d5 100644
--- a/ql/src/test/results/clientpositive/join34.q.out
+++ b/ql/src/test/results/clientpositive/join34.q.out
@@ -32,6 +32,17 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src
 POSTHOOK: Input: default@src1
 POSTHOOK: Output: default@dest_j1_n1
+OPTIMIZED SQL: SELECT `t5`.`key`, `t5`.`value`, `t3`.`value` AS `value1`
+FROM (SELECT `key`, `value`
+FROM `default`.`src`
+WHERE `key` < 20 AND CAST(`key` AS DOUBLE) BETWEEN 20 AND 100
+UNION ALL
+SELECT `key`, `value`
+FROM `default`.`src`
+WHERE `key` > 100 AND CAST(`key` AS DOUBLE) BETWEEN 20 AND 100) AS `t3`
+INNER JOIN (SELECT `key`, `value`
+FROM `default`.`src1`
+WHERE CAST(`key` AS DOUBLE) BETWEEN 20 AND 100 AND `key` IS NOT NULL) AS `t5` ON `t3`.`key` = `t5`.`key`
 STAGE DEPENDENCIES:
   Stage-7 is a root stage
   Stage-6 depends on stages: Stage-7
diff --git a/ql/src/test/results/clientpositive/join35.q.out b/ql/src/test/results/clientpositive/join35.q.out
index 3ef4146..f9485b3 100644
--- a/ql/src/test/results/clientpositive/join35.q.out
+++ b/ql/src/test/results/clientpositive/join35.q.out
@@ -32,6 +32,19 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src
 POSTHOOK: Input: default@src1
 POSTHOOK: Output: default@dest_j1_n24
+OPTIMIZED SQL: SELECT `t5`.`key`, `t5`.`value`, `t3`.`$f1` AS `cnt`
+FROM (SELECT `key`, COUNT(*) AS `$f1`
+FROM `default`.`src`
+WHERE `key` < 20 AND CAST(`key` AS DOUBLE) BETWEEN 20 AND 100
+GROUP BY `key`
+UNION ALL
+SELECT `key`, COUNT(*) AS `$f1`
+FROM `default`.`src`
+WHERE `key` > 100 AND CAST(`key` AS DOUBLE) BETWEEN 20 AND 100
+GROUP BY `key`) AS `t3`
+INNER JOIN (SELECT `key`, `value`
+FROM `default`.`src1`
+WHERE CAST(`key` AS DOUBLE) BETWEEN 20 AND 100 AND `key` IS NOT NULL) AS `t5` ON `t3`.`key` = `t5`.`key`
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
   Stage-8 depends on stages: Stage-1, Stage-5
diff --git a/ql/src/test/results/clientpositive/llap/external_jdbc_table2.q.out b/ql/src/test/results/clientpositive/llap/external_jdbc_table2.q.out
index a163577..ef68c7a 100644
--- a/ql/src/test/results/clientpositive/llap/external_jdbc_table2.q.out
+++ b/ql/src/test/results/clientpositive/llap/external_jdbc_table2.q.out
@@ -391,13 +391,13 @@ STAGE PLANS:
         TableScan
           alias: db1_ext_auth1
           properties:
-            hive.sql.query SELECT "t"."ikey", "t"."bkey", "t"."fkey", "t"."dkey", "t0"."ikey" AS "ikey0", "t0"."bkey" AS "bkey0", "t0"."fkey" AS "fkey0", "t0"."dkey" AS "dkey0"
-FROM (SELECT *
+            hive.sql.query SELECT "t0"."ikey", "t0"."bkey", "t0"."fkey", "t0"."dkey", "t2"."ikey" AS "ikey0", "t2"."bkey" AS "bkey0", "t2"."fkey" AS "fkey0", "t2"."dkey" AS "dkey0"
+FROM (SELECT "ikey", "bkey", "fkey", "dkey"
 FROM "EXTERNAL_JDBC_SIMPLE_DERBY2_TABLE1"
-WHERE "ikey" IS NOT NULL) AS "t"
-INNER JOIN (SELECT *
+WHERE "ikey" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "ikey", "bkey", "fkey", "dkey"
 FROM "EXTERNAL_JDBC_SIMPLE_DERBY2_TABLE2"
-WHERE "ikey" IS NOT NULL) AS "t0" ON "t"."ikey" = "t0"."ikey"
+WHERE "ikey" IS NOT NULL) AS "t2" ON "t0"."ikey" = "t2"."ikey"
             hive.sql.query.fieldNames ikey,bkey,fkey,dkey,ikey0,bkey0,fkey0,dkey0
             hive.sql.query.fieldTypes int,bigint,float,double,int,bigint,float,double
             hive.sql.query.split false
@@ -645,11 +645,11 @@ STAGE PLANS:
           alias: db1_ext_auth1
           properties:
             hive.sql.query SELECT "ikey", "bkey", "fkey", "dkey"
-FROM (SELECT *
+FROM (SELECT "ikey", "bkey", "fkey", "dkey"
 FROM "EXTERNAL_JDBC_SIMPLE_DERBY2_TABLE1"
 UNION ALL
-SELECT *
-FROM "EXTERNAL_JDBC_SIMPLE_DERBY2_TABLE2") AS "t"
+SELECT "ikey", "bkey", "fkey", "dkey"
+FROM "EXTERNAL_JDBC_SIMPLE_DERBY2_TABLE2") AS "t1"
             hive.sql.query.fieldNames ikey,bkey,fkey,dkey
             hive.sql.query.fieldTypes int,bigint,float,double
             hive.sql.query.split false
@@ -677,6 +677,138 @@ POSTHOOK: Input: default@db1_ext_auth2
 40	50	-455.4543	330.767
 44	53	-455.454	330.76
 PREHOOK: query: EXPLAIN
+SELECT P.ikey
+FROM
+  db1_ext_auth1 P
+WHERE
+  NOT TRUE OR P.`ikey` IS NOT NULL
+  AND (P.`bkey` = 10 AND P.`dkey` = 15.15
+    OR P.`bkey` = 20 AND P.`dkey` = 25.25)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@db1_ext_auth1
+#### A masked pattern was here ####
+POSTHOOK: query: EXPLAIN
+SELECT P.ikey
+FROM
+  db1_ext_auth1 P
+WHERE
+  NOT TRUE OR P.`ikey` IS NOT NULL
+  AND (P.`bkey` = 10 AND P.`dkey` = 15.15
+    OR P.`bkey` = 20 AND P.`dkey` = 25.25)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@db1_ext_auth1
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: p
+          properties:
+            hive.sql.query SELECT "ikey"
+FROM "EXTERNAL_JDBC_SIMPLE_DERBY2_TABLE1"
+WHERE "bkey" IN (10, 20) AND "dkey" IN (15.15, 25.25) AND ("bkey" = 10 AND "dkey" = 15.15 OR "bkey" = 20 AND "dkey" = 25.25) AND "ikey" IS NOT NULL
+            hive.sql.query.fieldNames ikey
+            hive.sql.query.fieldTypes int
+            hive.sql.query.split true
+          Select Operator
+            expressions: ikey (type: int)
+            outputColumnNames: _col0
+            ListSink
+
+PREHOOK: query: SELECT P.ikey
+FROM
+  db1_ext_auth1 P
+WHERE
+  NOT TRUE OR P.`ikey` IS NOT NULL
+  AND (P.`bkey` = 10 AND P.`dkey` = 15.15
+    OR P.`bkey` = 20 AND P.`dkey` = 25.25)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@db1_ext_auth1
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT P.ikey
+FROM
+  db1_ext_auth1 P
+WHERE
+  NOT TRUE OR P.`ikey` IS NOT NULL
+  AND (P.`bkey` = 10 AND P.`dkey` = 15.15
+    OR P.`bkey` = 20 AND P.`dkey` = 25.25)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@db1_ext_auth1
+#### A masked pattern was here ####
+PREHOOK: query: EXPLAIN
+SELECT P.ikey
+FROM
+  db1_ext_auth1 P
+WHERE
+  NOT TRUE OR P.`ikey` IS NOT NULL
+  AND (P.`bkey` = 10 AND P.`dkey` = 15.15
+    OR P.`bkey` = 20 AND P.`dkey` = 25.25)
+  AND fkey = null
+PREHOOK: type: QUERY
+PREHOOK: Input: default@db1_ext_auth1
+#### A masked pattern was here ####
+POSTHOOK: query: EXPLAIN
+SELECT P.ikey
+FROM
+  db1_ext_auth1 P
+WHERE
+  NOT TRUE OR P.`ikey` IS NOT NULL
+  AND (P.`bkey` = 10 AND P.`dkey` = 15.15
+    OR P.`bkey` = 20 AND P.`dkey` = 25.25)
+  AND fkey = null
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@db1_ext_auth1
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: p
+          properties:
+            hive.sql.query SELECT "ikey"
+FROM "EXTERNAL_JDBC_SIMPLE_DERBY2_TABLE1"
+WHERE FALSE
+            hive.sql.query.fieldNames ikey
+            hive.sql.query.fieldTypes int
+            hive.sql.query.split true
+          Select Operator
+            expressions: ikey (type: int)
+            outputColumnNames: _col0
+            ListSink
+
+PREHOOK: query: SELECT P.ikey
+FROM
+  db1_ext_auth1 P
+WHERE
+  NOT TRUE OR P.`ikey` IS NOT NULL
+  AND (P.`bkey` = 10 AND P.`dkey` = 15.15
+    OR P.`bkey` = 20 AND P.`dkey` = 25.25)
+  AND fkey = null
+PREHOOK: type: QUERY
+PREHOOK: Input: default@db1_ext_auth1
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT P.ikey
+FROM
+  db1_ext_auth1 P
+WHERE
+  NOT TRUE OR P.`ikey` IS NOT NULL
+  AND (P.`bkey` = 10 AND P.`dkey` = 15.15
+    OR P.`bkey` = 20 AND P.`dkey` = 25.25)
+  AND fkey = null
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@db1_ext_auth1
+#### A masked pattern was here ####
+PREHOOK: query: EXPLAIN
 SELECT * FROM db1_ext_auth1 UNION ALL SELECT * FROM db1_ext_auth2
 PREHOOK: type: QUERY
 PREHOOK: Input: default@db1_ext_auth1
diff --git a/ql/src/test/results/clientpositive/llap/external_jdbc_table4.q.out b/ql/src/test/results/clientpositive/llap/external_jdbc_table4.q.out
index ece0a6f..29866c2 100644
--- a/ql/src/test/results/clientpositive/llap/external_jdbc_table4.q.out
+++ b/ql/src/test/results/clientpositive/llap/external_jdbc_table4.q.out
@@ -391,13 +391,13 @@ STAGE PLANS:
         TableScan
           alias: db1_ext_auth1
           properties:
-            hive.sql.query SELECT "t"."IKEY", "t"."bkey", "t"."fkey", "t"."dkey", "t0"."ikey" AS "ikey0", "t0"."bkey" AS "bkey0", "t0"."FKEY" AS "FKEY0", "t0"."DKEY" AS "DKEY0"
-FROM (SELECT *
+            hive.sql.query SELECT "t0"."IKEY", "t0"."bkey", "t0"."fkey", "t0"."dkey", "t2"."ikey" AS "ikey0", "t2"."bkey" AS "bkey0", "t2"."FKEY" AS "FKEY0", "t2"."DKEY" AS "DKEY0"
+FROM (SELECT "IKEY", "bkey", "fkey", "dkey"
 FROM "EXTERNAL_JDBC_SIMPLE_DERBY2_TABLE1"
-WHERE "IKEY" IS NOT NULL) AS "t"
-INNER JOIN (SELECT *
+WHERE "IKEY" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "ikey", "bkey", "FKEY", "DKEY"
 FROM "EXTERNAL_JDBC_SIMPLE_DERBY2_TABLE2"
-WHERE "ikey" IS NOT NULL) AS "t0" ON "t"."IKEY" = "t0"."ikey"
+WHERE "ikey" IS NOT NULL) AS "t2" ON "t0"."IKEY" = "t2"."ikey"
             hive.sql.query.fieldNames IKEY,bkey,fkey,dkey,ikey0,bkey0,FKEY0,DKEY0
             hive.sql.query.fieldTypes int,bigint,float,double,int,bigint,float,double
             hive.sql.query.split false
diff --git a/ql/src/test/results/clientpositive/llap/external_jdbc_table_perf.q.out b/ql/src/test/results/clientpositive/llap/external_jdbc_table_perf.q.out
new file mode 100644
index 0000000..c1314c6
--- /dev/null
+++ b/ql/src/test/results/clientpositive/llap/external_jdbc_table_perf.q.out
@@ -0,0 +1,6230 @@
+PREHOOK: query: CREATE TEMPORARY FUNCTION dboutput AS 'org.apache.hadoop.hive.contrib.genericudf.example.GenericUDFDBOutput'
+PREHOOK: type: CREATEFUNCTION
+PREHOOK: Output: dboutput
+POSTHOOK: query: CREATE TEMPORARY FUNCTION dboutput AS 'org.apache.hadoop.hive.contrib.genericudf.example.GenericUDFDBOutput'
+POSTHOOK: type: CREATEFUNCTION
+POSTHOOK: Output: dboutput
+PREHOOK: query: FROM src
+SELECT
+#### A masked pattern was here ####
+'CREATE TABLE CATALOG_SALES ("cs_sold_date_sk" INTEGER, "cs_sold_time_sk" INTEGER, "cs_ship_date_sk" INTEGER, "cs_bill_customer_sk" INTEGER,
+  "cs_bill_cdemo_sk" INTEGER, "cs_bill_hdemo_sk" INTEGER, "cs_bill_addr_sk" INTEGER, "cs_ship_customer_sk" INTEGER, "cs_ship_cdemo_sk" INTEGER,
+  "cs_ship_hdemo_sk" INTEGER, "cs_ship_addr_sk" INTEGER, "cs_call_center_sk" INTEGER, "cs_catalog_page_sk" INTEGER, "cs_ship_mode_sk" INTEGER,
+  "cs_warehouse_sk" INTEGER, "cs_item_sk" INTEGER, "cs_promo_sk" INTEGER, "cs_order_number" INTEGER, "cs_quantity" INTEGER, "cs_wholesale_cost" DECIMAL(7,2),
+  "cs_list_price" DECIMAL(7,2), "cs_sales_price" DECIMAL(7,2), "cs_ext_discount_amt" DECIMAL(7,2), "cs_ext_sales_price" DECIMAL(7,2),
+  "cs_ext_wholesale_cost" DECIMAL(7,2), "cs_ext_list_price" DECIMAL(7,2), "cs_ext_tax" DECIMAL(7,2), "cs_coupon_amt" DECIMAL(7,2),
+  "cs_ext_ship_cost" DECIMAL(7,2), "cs_net_paid" DECIMAL(7,2), "cs_net_paid_inc_tax" DECIMAL(7,2), "cs_net_paid_inc_ship" DECIMAL(7,2),
+  "cs_net_paid_inc_ship_tax" DECIMAL(7,2), "cs_net_profit" DECIMAL(7,2))' ),
+#### A masked pattern was here ####
+'CREATE TABLE CATALOG_RETURNS ("cr_returned_date_sk" INTEGER, "cr_returned_time_sk" INTEGER, "cr_item_sk" INTEGER, "cr_refunded_customer_sk" INTEGER,
+  "cr_refunded_cdemo_sk" INTEGER, "cr_refunded_hdemo_sk" INTEGER, "cr_refunded_addr_sk" INTEGER, "cr_returning_customer_sk" INTEGER,
+  "cr_returning_cdemo_sk" INTEGER, "cr_returning_hdemo_sk" INTEGER, "cr_returning_addr_sk" INTEGER, "cr_call_center_sk" INTEGER,
+  "cr_catalog_page_sk" INTEGER, "cr_ship_mode_sk" INTEGER, "cr_warehouse_sk" INTEGER, "cr_reason_sk" INTEGER, "cr_order_number" INTEGER,
+  "cr_return_quantity" INTEGER, "cr_return_amount" DECIMAL(7,2), "cr_return_tax" DECIMAL(7,2), "cr_return_amt_inc_tax" DECIMAL(7,2),
+  "cr_fee" DECIMAL(7,2), "cr_return_ship_cost" DECIMAL(7,2), "cr_refunded_cash" DECIMAL(7,2), "cr_reversed_charge" DECIMAL(7,2),
+  "cr_store_credit" DECIMAL(7,2), "cr_net_loss" DECIMAL(7,2))' ),
+#### A masked pattern was here ####
+'CREATE TABLE STORE_SALES ("ss_sold_date_sk" INTEGER, "ss_sold_time_sk" INTEGER, "ss_item_sk" INTEGER, "ss_customer_sk" INTEGER, "ss_cdemo_sk" INTEGER,
+  "ss_hdemo_sk" INTEGER, "ss_addr_sk" INTEGER, "ss_store_sk" INTEGER, "ss_promo_sk" INTEGER, "ss_ticket_number" INTEGER, "ss_quantity" INTEGER,
+  "ss_wholesale_cost" DECIMAL(7,2), "ss_list_price" DECIMAL(7,2), "ss_sales_price" DECIMAL(7,2), "ss_ext_discount_amt" DECIMAL(7,2),
+  "ss_ext_sales_price" DECIMAL(7,2), "ss_ext_wholesale_cost" DECIMAL(7,2), "ss_ext_list_price" DECIMAL(7,2), "ss_ext_tax" DECIMAL(7,2),
+  "ss_coupon_amt" DECIMAL(7,2), "ss_net_paid" DECIMAL(7,2), "ss_net_paid_inc_tax" DECIMAL(7,2), "ss_net_profit" DECIMAL(7,2))' ),
+#### A masked pattern was here ####
+'CREATE TABLE STORE_RETURNS ("sr_returned_date_sk" INTEGER, "sr_return_time_sk" INTEGER, "sr_item_sk" INTEGER, "sr_customer_sk" INTEGER,
+  "sr_cdemo_sk" INTEGER, "sr_hdemo_sk" INTEGER, "sr_addr_sk" INTEGER, "sr_store_sk" INTEGER, "sr_reason_sk" INTEGER, "sr_ticket_number" INTEGER,
+  "sr_return_quantity" INTEGER, "sr_return_amt" DECIMAL(7,2), "sr_return_tax" DECIMAL(7,2), "sr_return_amt_inc_tax" DECIMAL(7,2),
+  "sr_fee" DECIMAL(7,2), "sr_return_ship_cost" DECIMAL(7,2), "sr_refunded_cash" DECIMAL(7,2), "sr_reversed_charge" DECIMAL(7,2),
+  "sr_store_credit" DECIMAL(7,2), "sr_net_loss" DECIMAL(7,2))' ),
+#### A masked pattern was here ####
+'CREATE TABLE WEB_SALES ("ws_sold_date_sk" INTEGER, "ws_sold_time_sk" INTEGER, "ws_ship_date_sk" INTEGER, "ws_item_sk" INTEGER,
+  "ws_bill_customer_sk" INTEGER, "ws_bill_cdemo_sk" INTEGER, "ws_bill_hdemo_sk" INTEGER, "ws_bill_addr_sk" INTEGER, "ws_ship_customer_sk" INTEGER,
+  "ws_ship_cdemo_sk" INTEGER, "ws_ship_hdemo_sk" INTEGER, "ws_ship_addr_sk" INTEGER, "ws_web_page_sk" INTEGER, "ws_web_site_sk" INTEGER,
+  "ws_ship_mode_sk" INTEGER, "ws_warehouse_sk" INTEGER, "ws_promo_sk" INTEGER, "ws_order_number" INTEGER, "ws_quantity" INTEGER,
+  "ws_wholesale_cost" DECIMAL(7,2), "ws_list_price" DECIMAL(7,2), "ws_sales_price" DECIMAL(7,2), "ws_ext_discount_amt" DECIMAL(7,2),
+  "ws_ext_sales_price" DECIMAL(7,2), "ws_ext_wholesale_cost" DECIMAL(7,2), "ws_ext_list_price" DECIMAL(7,2), "ws_ext_tax" DECIMAL(7,2),
+  "ws_coupon_amt" DECIMAL(7,2), "ws_ext_ship_cost" DECIMAL(7,2), "ws_net_paid" DECIMAL(7,2), "ws_net_paid_inc_tax" DECIMAL(7,2),
+  "ws_net_paid_inc_ship" DECIMAL(7,2), "ws_net_paid_inc_ship_tax" DECIMAL(7,2), "ws_net_profit" DECIMAL(7,2))' ),
+#### A masked pattern was here ####
+'CREATE TABLE WEB_RETURNS ("wr_returned_date_sk" INTEGER, "wr_returned_time_sk" INTEGER, "wr_item_sk" INTEGER, "wr_refunded_customer_sk" INTEGER,
+  "wr_refunded_cdemo_sk" INTEGER, "wr_refunded_hdemo_sk" INTEGER, "wr_refunded_addr_sk" INTEGER, "wr_returning_customer_sk" INTEGER,
+  "wr_returning_cdemo_sk" INTEGER, "wr_returning_hdemo_sk" INTEGER, "wr_returning_addr_sk" INTEGER, "wr_web_page_sk" INTEGER,
+  "wr_reason_sk" INTEGER, "wr_order_number" INTEGER, "wr_return_quantity" INTEGER, "wr_return_amt" DECIMAL(7,2), "wr_return_tax" DECIMAL(7,2),
+  "wr_return_amt_inc_tax" DECIMAL(7,2), "wr_fee" DECIMAL(7,2), "wr_return_ship_cost" DECIMAL(7,2), "wr_refunded_cash" DECIMAL(7,2),
+  "wr_reversed_charge" DECIMAL(7,2), "wr_account_credit" DECIMAL(7,2), "wr_net_loss" DECIMAL(7,2))' ),
+#### A masked pattern was here ####
+'CREATE TABLE CUSTOMER ("c_customer_sk" INTEGER, "c_customer_id" CHAR(16), "c_current_cdemo_sk" INTEGER, "c_current_hdemo_sk" INTEGER,
+  "c_current_addr_sk" INTEGER, "c_first_shipto_date_sk" INTEGER, "c_first_sales_date_sk" INTEGER, "c_salutation" CHAR(10),
+  "c_first_name" CHAR(20), "c_last_name" CHAR(30), "c_preferred_cust_flag" CHAR(1), "c_birth_day" INTEGER, "c_birth_month" INTEGER,
+  "c_birth_year" INTEGER, "c_birth_country" VARCHAR(20), "c_login" CHAR(13), "c_email_address" CHAR(50), "c_last_review_date" CHAR(10))' ),
+#### A masked pattern was here ####
+'CREATE TABLE CUSTOMER_ADDRESS ("ca_address_sk" INTEGER, "ca_address_id" CHAR(16), "ca_street_number" CHAR(10), "ca_street_name" VARCHAR(60),
+  "ca_street_type" CHAR(15), "ca_suite_number" CHAR(10), "ca_city" VARCHAR(60), "ca_county" VARCHAR(30), "ca_state" CHAR(2),
+  "ca_zip" CHAR(10), "ca_country" VARCHAR(20), "ca_gmt_offset" DECIMAL(5,2), "ca_location_type" CHAR(20))' ),
+#### A masked pattern was here ####
+'CREATE TABLE CUSTOMER_DEMOGRAPHICS ("cd_demo_sk" INTEGER, "cd_gender" CHAR(1), "cd_marital_status" CHAR(1), "cd_education_status" CHAR(20),
+  "cd_purchase_estimate" INTEGER, "cd_credit_rating" CHAR(10), "cd_dep_count" INTEGER, "cd_dep_employed_count" INTEGER,
+  "cd_dep_college_count" INTEGER)' ),
+#### A masked pattern was here ####
+'CREATE TABLE INVENTORY ("inv_date_sk" INTEGER, "inv_item_sk" INTEGER, "inv_warehouse_sk" INTEGER, "inv_quantity_on_hand" INTEGER)' ),
+#### A masked pattern was here ####
+'CREATE TABLE ITEM ("i_item_sk" INTEGER, "i_item_id" CHAR(16), "i_rec_start_date" DATE, "i_rec_end_date" DATE,
+  "i_item_desc" VARCHAR(200), "i_current_price" DECIMAL(7,2), "i_wholesale_cost" DECIMAL(7,2), "i_brand_id" INTEGER,
+  "i_brand" CHAR(50), "i_class_id" INTEGER, "i_class" CHAR(50), "i_category_id" INTEGER, "i_category" CHAR(50),
+  "i_manufact_id" INTEGER, "i_manufact" CHAR(50), "i_size" CHAR(20), "i_formulation" CHAR(20), "i_color" CHAR(20),
+  "i_units" CHAR(10), "i_container" CHAR(10), "i_manager_id" INTEGER, "i_product_name" CHAR(50))' ),
+#### A masked pattern was here ####
+'CREATE TABLE WAREHOUSE ("w_warehouse_sk" INTEGER, "w_warehouse_id" CHAR(16), "w_warehouse_name" VARCHAR(20),
+  "w_warehouse_sq_ft" INTEGER, "w_street_number" CHAR(10), "w_street_name" VARCHAR(60), "w_street_type" CHAR(15),
+  "w_suite_number" CHAR(10), "w_city" VARCHAR(60), "w_county" VARCHAR(30), "w_state" CHAR(2), "w_zip" CHAR(10),
+  "w_country" VARCHAR(20), "w_gmt_offset" DECIMAL(5,2))' ),
+#### A masked pattern was here ####
+'CREATE TABLE DATE_DIM ("d_date_sk" INTEGER, "d_date_id" CHAR(16), "d_date" DATE, "d_month_seq" INTEGER, "d_week_seq" INTEGER,
+  "d_quarter_seq" INTEGER, "d_year" INTEGER, "d_dow" INTEGER, "d_moy" INTEGER, "d_dom" INTEGER, "d_qoy" INTEGER,
+  "d_fy_year" INTEGER, "d_fy_quarter_seq" INTEGER, "d_fy_week_seq" INTEGER, "d_day_name" CHAR(9), "d_quarter_name" CHAR(6),
+  "d_holiday" CHAR(1), "d_weekend" CHAR(1), "d_following_holiday" CHAR(1), "d_first_dom" INTEGER, "d_last_dom" INTEGER,
+  "d_same_day_ly" INTEGER, "d_same_day_lq" INTEGER, "d_current_day" CHAR(1), "d_current_week" CHAR(1), "d_current_month" CHAR(1),
+  "d_current_quarter" CHAR(1), "d_current_year" CHAR(1))' ),
+#### A masked pattern was here ####
+'CREATE TABLE STORE ("s_store_sk" INTEGER, "s_store_id" CHAR(16), "s_rec_start_date" DATE, "s_rec_end_date" DATE,
+  "s_closed_date_sk" INTEGER, "s_store_name" VARCHAR(50), "s_number_employees" INTEGER, "s_floor_space" INTEGER,
+  "s_hours" CHAR(20), "s_manager" VARCHAR(40), "s_market_id" INTEGER, "s_geography_class" VARCHAR(100),
+  "s_market_desc" VARCHAR(100), "s_market_manager" VARCHAR(40), "s_division_id" INTEGER, "s_division_name" VARCHAR(50),
+  "s_company_id" INTEGER, "s_company_name" VARCHAR(50), "s_street_number" VARCHAR(10), "s_street_name" VARCHAR(60),
+  "s_street_type" CHAR(15), "s_suite_number" CHAR(10), "s_city" VARCHAR(60), "s_county" VARCHAR(30),
+  "s_state" CHAR(2), "s_zip" CHAR(10), "s_country" VARCHAR(20), "s_gmt_offset" DECIMAL(5,2), "s_tax_precentage" DECIMAL(5,2))' ),
+#### A masked pattern was here ####
+'CREATE TABLE HOUSEHOLD_DEMOGRAPHICS ("hd_demo_sk" INTEGER, "hd_income_band_sk" INTEGER, "hd_buy_potential" CHAR(15),
+  "hd_dep_count" INTEGER, "hd_vehicle_count" INTEGER)' ),
+#### A masked pattern was here ####
+'CREATE TABLE TIME_DIM ("t_time_sk" INTEGER, "t_time_id" CHAR(16), "t_time" INTEGER, "t_hour" INTEGER,
+  "t_minute" INTEGER, "t_second" INTEGER, "t_am_pm" CHAR(2), "t_shift" CHAR(20), "t_sub_shift" CHAR(20),
+  "t_meal_time" CHAR(20))' ),
+#### A masked pattern was here ####
+'CREATE TABLE REASON ("r_reason_sk" INTEGER, "r_reason_id" CHAR(16), "r_reason_desc" CHAR(100))' )
+limit 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: FROM src
+SELECT
+#### A masked pattern was here ####
+'CREATE TABLE CATALOG_SALES ("cs_sold_date_sk" INTEGER, "cs_sold_time_sk" INTEGER, "cs_ship_date_sk" INTEGER, "cs_bill_customer_sk" INTEGER,
+  "cs_bill_cdemo_sk" INTEGER, "cs_bill_hdemo_sk" INTEGER, "cs_bill_addr_sk" INTEGER, "cs_ship_customer_sk" INTEGER, "cs_ship_cdemo_sk" INTEGER,
+  "cs_ship_hdemo_sk" INTEGER, "cs_ship_addr_sk" INTEGER, "cs_call_center_sk" INTEGER, "cs_catalog_page_sk" INTEGER, "cs_ship_mode_sk" INTEGER,
+  "cs_warehouse_sk" INTEGER, "cs_item_sk" INTEGER, "cs_promo_sk" INTEGER, "cs_order_number" INTEGER, "cs_quantity" INTEGER, "cs_wholesale_cost" DECIMAL(7,2),
+  "cs_list_price" DECIMAL(7,2), "cs_sales_price" DECIMAL(7,2), "cs_ext_discount_amt" DECIMAL(7,2), "cs_ext_sales_price" DECIMAL(7,2),
+  "cs_ext_wholesale_cost" DECIMAL(7,2), "cs_ext_list_price" DECIMAL(7,2), "cs_ext_tax" DECIMAL(7,2), "cs_coupon_amt" DECIMAL(7,2),
+  "cs_ext_ship_cost" DECIMAL(7,2), "cs_net_paid" DECIMAL(7,2), "cs_net_paid_inc_tax" DECIMAL(7,2), "cs_net_paid_inc_ship" DECIMAL(7,2),
+  "cs_net_paid_inc_ship_tax" DECIMAL(7,2), "cs_net_profit" DECIMAL(7,2))' ),
+#### A masked pattern was here ####
+'CREATE TABLE CATALOG_RETURNS ("cr_returned_date_sk" INTEGER, "cr_returned_time_sk" INTEGER, "cr_item_sk" INTEGER, "cr_refunded_customer_sk" INTEGER,
+  "cr_refunded_cdemo_sk" INTEGER, "cr_refunded_hdemo_sk" INTEGER, "cr_refunded_addr_sk" INTEGER, "cr_returning_customer_sk" INTEGER,
+  "cr_returning_cdemo_sk" INTEGER, "cr_returning_hdemo_sk" INTEGER, "cr_returning_addr_sk" INTEGER, "cr_call_center_sk" INTEGER,
+  "cr_catalog_page_sk" INTEGER, "cr_ship_mode_sk" INTEGER, "cr_warehouse_sk" INTEGER, "cr_reason_sk" INTEGER, "cr_order_number" INTEGER,
+  "cr_return_quantity" INTEGER, "cr_return_amount" DECIMAL(7,2), "cr_return_tax" DECIMAL(7,2), "cr_return_amt_inc_tax" DECIMAL(7,2),
+  "cr_fee" DECIMAL(7,2), "cr_return_ship_cost" DECIMAL(7,2), "cr_refunded_cash" DECIMAL(7,2), "cr_reversed_charge" DECIMAL(7,2),
+  "cr_store_credit" DECIMAL(7,2), "cr_net_loss" DECIMAL(7,2))' ),
+#### A masked pattern was here ####
+'CREATE TABLE STORE_SALES ("ss_sold_date_sk" INTEGER, "ss_sold_time_sk" INTEGER, "ss_item_sk" INTEGER, "ss_customer_sk" INTEGER, "ss_cdemo_sk" INTEGER,
+  "ss_hdemo_sk" INTEGER, "ss_addr_sk" INTEGER, "ss_store_sk" INTEGER, "ss_promo_sk" INTEGER, "ss_ticket_number" INTEGER, "ss_quantity" INTEGER,
+  "ss_wholesale_cost" DECIMAL(7,2), "ss_list_price" DECIMAL(7,2), "ss_sales_price" DECIMAL(7,2), "ss_ext_discount_amt" DECIMAL(7,2),
+  "ss_ext_sales_price" DECIMAL(7,2), "ss_ext_wholesale_cost" DECIMAL(7,2), "ss_ext_list_price" DECIMAL(7,2), "ss_ext_tax" DECIMAL(7,2),
+  "ss_coupon_amt" DECIMAL(7,2), "ss_net_paid" DECIMAL(7,2), "ss_net_paid_inc_tax" DECIMAL(7,2), "ss_net_profit" DECIMAL(7,2))' ),
+#### A masked pattern was here ####
+'CREATE TABLE STORE_RETURNS ("sr_returned_date_sk" INTEGER, "sr_return_time_sk" INTEGER, "sr_item_sk" INTEGER, "sr_customer_sk" INTEGER,
+  "sr_cdemo_sk" INTEGER, "sr_hdemo_sk" INTEGER, "sr_addr_sk" INTEGER, "sr_store_sk" INTEGER, "sr_reason_sk" INTEGER, "sr_ticket_number" INTEGER,
+  "sr_return_quantity" INTEGER, "sr_return_amt" DECIMAL(7,2), "sr_return_tax" DECIMAL(7,2), "sr_return_amt_inc_tax" DECIMAL(7,2),
+  "sr_fee" DECIMAL(7,2), "sr_return_ship_cost" DECIMAL(7,2), "sr_refunded_cash" DECIMAL(7,2), "sr_reversed_charge" DECIMAL(7,2),
+  "sr_store_credit" DECIMAL(7,2), "sr_net_loss" DECIMAL(7,2))' ),
+#### A masked pattern was here ####
+'CREATE TABLE WEB_SALES ("ws_sold_date_sk" INTEGER, "ws_sold_time_sk" INTEGER, "ws_ship_date_sk" INTEGER, "ws_item_sk" INTEGER,
+  "ws_bill_customer_sk" INTEGER, "ws_bill_cdemo_sk" INTEGER, "ws_bill_hdemo_sk" INTEGER, "ws_bill_addr_sk" INTEGER, "ws_ship_customer_sk" INTEGER,
+  "ws_ship_cdemo_sk" INTEGER, "ws_ship_hdemo_sk" INTEGER, "ws_ship_addr_sk" INTEGER, "ws_web_page_sk" INTEGER, "ws_web_site_sk" INTEGER,
+  "ws_ship_mode_sk" INTEGER, "ws_warehouse_sk" INTEGER, "ws_promo_sk" INTEGER, "ws_order_number" INTEGER, "ws_quantity" INTEGER,
+  "ws_wholesale_cost" DECIMAL(7,2), "ws_list_price" DECIMAL(7,2), "ws_sales_price" DECIMAL(7,2), "ws_ext_discount_amt" DECIMAL(7,2),
+  "ws_ext_sales_price" DECIMAL(7,2), "ws_ext_wholesale_cost" DECIMAL(7,2), "ws_ext_list_price" DECIMAL(7,2), "ws_ext_tax" DECIMAL(7,2),
+  "ws_coupon_amt" DECIMAL(7,2), "ws_ext_ship_cost" DECIMAL(7,2), "ws_net_paid" DECIMAL(7,2), "ws_net_paid_inc_tax" DECIMAL(7,2),
+  "ws_net_paid_inc_ship" DECIMAL(7,2), "ws_net_paid_inc_ship_tax" DECIMAL(7,2), "ws_net_profit" DECIMAL(7,2))' ),
+#### A masked pattern was here ####
+'CREATE TABLE WEB_RETURNS ("wr_returned_date_sk" INTEGER, "wr_returned_time_sk" INTEGER, "wr_item_sk" INTEGER, "wr_refunded_customer_sk" INTEGER,
+  "wr_refunded_cdemo_sk" INTEGER, "wr_refunded_hdemo_sk" INTEGER, "wr_refunded_addr_sk" INTEGER, "wr_returning_customer_sk" INTEGER,
+  "wr_returning_cdemo_sk" INTEGER, "wr_returning_hdemo_sk" INTEGER, "wr_returning_addr_sk" INTEGER, "wr_web_page_sk" INTEGER,
+  "wr_reason_sk" INTEGER, "wr_order_number" INTEGER, "wr_return_quantity" INTEGER, "wr_return_amt" DECIMAL(7,2), "wr_return_tax" DECIMAL(7,2),
+  "wr_return_amt_inc_tax" DECIMAL(7,2), "wr_fee" DECIMAL(7,2), "wr_return_ship_cost" DECIMAL(7,2), "wr_refunded_cash" DECIMAL(7,2),
+  "wr_reversed_charge" DECIMAL(7,2), "wr_account_credit" DECIMAL(7,2), "wr_net_loss" DECIMAL(7,2))' ),
+#### A masked pattern was here ####
+'CREATE TABLE CUSTOMER ("c_customer_sk" INTEGER, "c_customer_id" CHAR(16), "c_current_cdemo_sk" INTEGER, "c_current_hdemo_sk" INTEGER,
+  "c_current_addr_sk" INTEGER, "c_first_shipto_date_sk" INTEGER, "c_first_sales_date_sk" INTEGER, "c_salutation" CHAR(10),
+  "c_first_name" CHAR(20), "c_last_name" CHAR(30), "c_preferred_cust_flag" CHAR(1), "c_birth_day" INTEGER, "c_birth_month" INTEGER,
+  "c_birth_year" INTEGER, "c_birth_country" VARCHAR(20), "c_login" CHAR(13), "c_email_address" CHAR(50), "c_last_review_date" CHAR(10))' ),
+#### A masked pattern was here ####
+'CREATE TABLE CUSTOMER_ADDRESS ("ca_address_sk" INTEGER, "ca_address_id" CHAR(16), "ca_street_number" CHAR(10), "ca_street_name" VARCHAR(60),
+  "ca_street_type" CHAR(15), "ca_suite_number" CHAR(10), "ca_city" VARCHAR(60), "ca_county" VARCHAR(30), "ca_state" CHAR(2),
+  "ca_zip" CHAR(10), "ca_country" VARCHAR(20), "ca_gmt_offset" DECIMAL(5,2), "ca_location_type" CHAR(20))' ),
+#### A masked pattern was here ####
+'CREATE TABLE CUSTOMER_DEMOGRAPHICS ("cd_demo_sk" INTEGER, "cd_gender" CHAR(1), "cd_marital_status" CHAR(1), "cd_education_status" CHAR(20),
+  "cd_purchase_estimate" INTEGER, "cd_credit_rating" CHAR(10), "cd_dep_count" INTEGER, "cd_dep_employed_count" INTEGER,
+  "cd_dep_college_count" INTEGER)' ),
+#### A masked pattern was here ####
+'CREATE TABLE INVENTORY ("inv_date_sk" INTEGER, "inv_item_sk" INTEGER, "inv_warehouse_sk" INTEGER, "inv_quantity_on_hand" INTEGER)' ),
+#### A masked pattern was here ####
+'CREATE TABLE ITEM ("i_item_sk" INTEGER, "i_item_id" CHAR(16), "i_rec_start_date" DATE, "i_rec_end_date" DATE,
+  "i_item_desc" VARCHAR(200), "i_current_price" DECIMAL(7,2), "i_wholesale_cost" DECIMAL(7,2), "i_brand_id" INTEGER,
+  "i_brand" CHAR(50), "i_class_id" INTEGER, "i_class" CHAR(50), "i_category_id" INTEGER, "i_category" CHAR(50),
+  "i_manufact_id" INTEGER, "i_manufact" CHAR(50), "i_size" CHAR(20), "i_formulation" CHAR(20), "i_color" CHAR(20),
+  "i_units" CHAR(10), "i_container" CHAR(10), "i_manager_id" INTEGER, "i_product_name" CHAR(50))' ),
+#### A masked pattern was here ####
+'CREATE TABLE WAREHOUSE ("w_warehouse_sk" INTEGER, "w_warehouse_id" CHAR(16), "w_warehouse_name" VARCHAR(20),
+  "w_warehouse_sq_ft" INTEGER, "w_street_number" CHAR(10), "w_street_name" VARCHAR(60), "w_street_type" CHAR(15),
+  "w_suite_number" CHAR(10), "w_city" VARCHAR(60), "w_county" VARCHAR(30), "w_state" CHAR(2), "w_zip" CHAR(10),
+  "w_country" VARCHAR(20), "w_gmt_offset" DECIMAL(5,2))' ),
+#### A masked pattern was here ####
+'CREATE TABLE DATE_DIM ("d_date_sk" INTEGER, "d_date_id" CHAR(16), "d_date" DATE, "d_month_seq" INTEGER, "d_week_seq" INTEGER,
+  "d_quarter_seq" INTEGER, "d_year" INTEGER, "d_dow" INTEGER, "d_moy" INTEGER, "d_dom" INTEGER, "d_qoy" INTEGER,
+  "d_fy_year" INTEGER, "d_fy_quarter_seq" INTEGER, "d_fy_week_seq" INTEGER, "d_day_name" CHAR(9), "d_quarter_name" CHAR(6),
+  "d_holiday" CHAR(1), "d_weekend" CHAR(1), "d_following_holiday" CHAR(1), "d_first_dom" INTEGER, "d_last_dom" INTEGER,
+  "d_same_day_ly" INTEGER, "d_same_day_lq" INTEGER, "d_current_day" CHAR(1), "d_current_week" CHAR(1), "d_current_month" CHAR(1),
+  "d_current_quarter" CHAR(1), "d_current_year" CHAR(1))' ),
+#### A masked pattern was here ####
+'CREATE TABLE STORE ("s_store_sk" INTEGER, "s_store_id" CHAR(16), "s_rec_start_date" DATE, "s_rec_end_date" DATE,
+  "s_closed_date_sk" INTEGER, "s_store_name" VARCHAR(50), "s_number_employees" INTEGER, "s_floor_space" INTEGER,
+  "s_hours" CHAR(20), "s_manager" VARCHAR(40), "s_market_id" INTEGER, "s_geography_class" VARCHAR(100),
+  "s_market_desc" VARCHAR(100), "s_market_manager" VARCHAR(40), "s_division_id" INTEGER, "s_division_name" VARCHAR(50),
+  "s_company_id" INTEGER, "s_company_name" VARCHAR(50), "s_street_number" VARCHAR(10), "s_street_name" VARCHAR(60),
+  "s_street_type" CHAR(15), "s_suite_number" CHAR(10), "s_city" VARCHAR(60), "s_county" VARCHAR(30),
+  "s_state" CHAR(2), "s_zip" CHAR(10), "s_country" VARCHAR(20), "s_gmt_offset" DECIMAL(5,2), "s_tax_precentage" DECIMAL(5,2))' ),
+#### A masked pattern was here ####
+'CREATE TABLE HOUSEHOLD_DEMOGRAPHICS ("hd_demo_sk" INTEGER, "hd_income_band_sk" INTEGER, "hd_buy_potential" CHAR(15),
+  "hd_dep_count" INTEGER, "hd_vehicle_count" INTEGER)' ),
+#### A masked pattern was here ####
+'CREATE TABLE TIME_DIM ("t_time_sk" INTEGER, "t_time_id" CHAR(16), "t_time" INTEGER, "t_hour" INTEGER,
+  "t_minute" INTEGER, "t_second" INTEGER, "t_am_pm" CHAR(2), "t_shift" CHAR(20), "t_sub_shift" CHAR(20),
+  "t_meal_time" CHAR(20))' ),
+#### A masked pattern was here ####
+'CREATE TABLE REASON ("r_reason_sk" INTEGER, "r_reason_id" CHAR(16), "r_reason_desc" CHAR(100))' )
+limit 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+0	0	0	0	0	0	0	0	0	0	0	0	0	0	0	0	0
+PREHOOK: query: CREATE EXTERNAL TABLE catalog_sales
+(
+    cs_sold_date_sk           int                           ,
+    cs_sold_time_sk           int                           ,
+    cs_ship_date_sk           int                           ,
+    cs_bill_customer_sk       int                           ,
+    cs_bill_cdemo_sk          int                           ,
+    cs_bill_hdemo_sk          int                           ,
+    cs_bill_addr_sk           int                           ,
+    cs_ship_customer_sk       int                           ,
+    cs_ship_cdemo_sk          int                           ,
+    cs_ship_hdemo_sk          int                           ,
+    cs_ship_addr_sk           int                           ,
+    cs_call_center_sk         int                           ,
+    cs_catalog_page_sk        int                           ,
+    cs_ship_mode_sk           int                           ,
+    cs_warehouse_sk           int                           ,
+    cs_item_sk                int                           ,
+    cs_promo_sk               int                           ,
+    cs_order_number           int                           ,
+    cs_quantity               int                           ,
+    cs_wholesale_cost         decimal(7,2)                  ,
+    cs_list_price             decimal(7,2)                  ,
+    cs_sales_price            decimal(7,2)                  ,
+    cs_ext_discount_amt       decimal(7,2)                  ,
+    cs_ext_sales_price        decimal(7,2)                  ,
+    cs_ext_wholesale_cost     decimal(7,2)                  ,
+    cs_ext_list_price         decimal(7,2)                  ,
+    cs_ext_tax                decimal(7,2)                  ,
+    cs_coupon_amt             decimal(7,2)                  ,
+    cs_ext_ship_cost          decimal(7,2)                  ,
+    cs_net_paid               decimal(7,2)                  ,
+    cs_net_paid_inc_tax       decimal(7,2)                  ,
+    cs_net_paid_inc_ship      decimal(7,2)                  ,
+    cs_net_paid_inc_ship_tax  decimal(7,2)                  ,
+    cs_net_profit             decimal(7,2)
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+#### A masked pattern was here ####
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "CATALOG_SALES",
+                "hive.sql.dbcp.maxActive" = "1"
+)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@catalog_sales
+POSTHOOK: query: CREATE EXTERNAL TABLE catalog_sales
+(
+    cs_sold_date_sk           int                           ,
+    cs_sold_time_sk           int                           ,
+    cs_ship_date_sk           int                           ,
+    cs_bill_customer_sk       int                           ,
+    cs_bill_cdemo_sk          int                           ,
+    cs_bill_hdemo_sk          int                           ,
+    cs_bill_addr_sk           int                           ,
+    cs_ship_customer_sk       int                           ,
+    cs_ship_cdemo_sk          int                           ,
+    cs_ship_hdemo_sk          int                           ,
+    cs_ship_addr_sk           int                           ,
+    cs_call_center_sk         int                           ,
+    cs_catalog_page_sk        int                           ,
+    cs_ship_mode_sk           int                           ,
+    cs_warehouse_sk           int                           ,
+    cs_item_sk                int                           ,
+    cs_promo_sk               int                           ,
+    cs_order_number           int                           ,
+    cs_quantity               int                           ,
+    cs_wholesale_cost         decimal(7,2)                  ,
+    cs_list_price             decimal(7,2)                  ,
+    cs_sales_price            decimal(7,2)                  ,
+    cs_ext_discount_amt       decimal(7,2)                  ,
+    cs_ext_sales_price        decimal(7,2)                  ,
+    cs_ext_wholesale_cost     decimal(7,2)                  ,
+    cs_ext_list_price         decimal(7,2)                  ,
+    cs_ext_tax                decimal(7,2)                  ,
+    cs_coupon_amt             decimal(7,2)                  ,
+    cs_ext_ship_cost          decimal(7,2)                  ,
+    cs_net_paid               decimal(7,2)                  ,
+    cs_net_paid_inc_tax       decimal(7,2)                  ,
+    cs_net_paid_inc_ship      decimal(7,2)                  ,
+    cs_net_paid_inc_ship_tax  decimal(7,2)                  ,
+    cs_net_profit             decimal(7,2)
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+#### A masked pattern was here ####
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "CATALOG_SALES",
+                "hive.sql.dbcp.maxActive" = "1"
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@catalog_sales
+PREHOOK: query: CREATE EXTERNAL TABLE catalog_returns
+(
+    cr_returned_date_sk       int                           ,
+    cr_returned_time_sk       int                           ,
+    cr_item_sk                int                           ,
+    cr_refunded_customer_sk   int                           ,
+    cr_refunded_cdemo_sk      int                           ,
+    cr_refunded_hdemo_sk      int                           ,
+    cr_refunded_addr_sk       int                           ,
+    cr_returning_customer_sk  int                           ,
+    cr_returning_cdemo_sk     int                           ,
+    cr_returning_hdemo_sk     int                           ,
+    cr_returning_addr_sk      int                           ,
+    cr_call_center_sk         int                           ,
+    cr_catalog_page_sk        int                           ,
+    cr_ship_mode_sk           int                           ,
+    cr_warehouse_sk           int                           ,
+    cr_reason_sk              int                           ,
+    cr_order_number           int                           ,
+    cr_return_quantity        int                           ,
+    cr_return_amount          decimal(7,2)                  ,
+    cr_return_tax             decimal(7,2)                  ,
+    cr_return_amt_inc_tax     decimal(7,2)                  ,
+    cr_fee                    decimal(7,2)                  ,
+    cr_return_ship_cost       decimal(7,2)                  ,
+    cr_refunded_cash          decimal(7,2)                  ,
+    cr_reversed_charge        decimal(7,2)                  ,
+    cr_store_credit           decimal(7,2)                  ,
+    cr_net_loss               decimal(7,2)
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+#### A masked pattern was here ####
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "CATALOG_RETURNS",
+                "hive.sql.dbcp.maxActive" = "1"
+)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@catalog_returns
+POSTHOOK: query: CREATE EXTERNAL TABLE catalog_returns
+(
+    cr_returned_date_sk       int                           ,
+    cr_returned_time_sk       int                           ,
+    cr_item_sk                int                           ,
+    cr_refunded_customer_sk   int                           ,
+    cr_refunded_cdemo_sk      int                           ,
+    cr_refunded_hdemo_sk      int                           ,
+    cr_refunded_addr_sk       int                           ,
+    cr_returning_customer_sk  int                           ,
+    cr_returning_cdemo_sk     int                           ,
+    cr_returning_hdemo_sk     int                           ,
+    cr_returning_addr_sk      int                           ,
+    cr_call_center_sk         int                           ,
+    cr_catalog_page_sk        int                           ,
+    cr_ship_mode_sk           int                           ,
+    cr_warehouse_sk           int                           ,
+    cr_reason_sk              int                           ,
+    cr_order_number           int                           ,
+    cr_return_quantity        int                           ,
+    cr_return_amount          decimal(7,2)                  ,
+    cr_return_tax             decimal(7,2)                  ,
+    cr_return_amt_inc_tax     decimal(7,2)                  ,
+    cr_fee                    decimal(7,2)                  ,
+    cr_return_ship_cost       decimal(7,2)                  ,
+    cr_refunded_cash          decimal(7,2)                  ,
+    cr_reversed_charge        decimal(7,2)                  ,
+    cr_store_credit           decimal(7,2)                  ,
+    cr_net_loss               decimal(7,2)
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+#### A masked pattern was here ####
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "CATALOG_RETURNS",
+                "hive.sql.dbcp.maxActive" = "1"
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@catalog_returns
+PREHOOK: query: CREATE EXTERNAL TABLE store_sales
+(
+    ss_sold_date_sk           int                           ,
+    ss_sold_time_sk           int                           ,
+    ss_item_sk                int                           ,
+    ss_customer_sk            int                           ,
+    ss_cdemo_sk               int                           ,
+    ss_hdemo_sk               int                           ,
+    ss_addr_sk                int                           ,
+    ss_store_sk               int                           ,
+    ss_promo_sk               int                           ,
+    ss_ticket_number          int                           ,
+    ss_quantity               int                           ,
+    ss_wholesale_cost         decimal(7,2)                  ,
+    ss_list_price             decimal(7,2)                  ,
+    ss_sales_price            decimal(7,2)                  ,
+    ss_ext_discount_amt       decimal(7,2)                  ,
+    ss_ext_sales_price        decimal(7,2)                  ,
+    ss_ext_wholesale_cost     decimal(7,2)                  ,
+    ss_ext_list_price         decimal(7,2)                  ,
+    ss_ext_tax                decimal(7,2)                  ,
+    ss_coupon_amt             decimal(7,2)                  ,
+    ss_net_paid               decimal(7,2)                  ,
+    ss_net_paid_inc_tax       decimal(7,2)                  ,
+    ss_net_profit             decimal(7,2)
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+#### A masked pattern was here ####
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "STORE_SALES",
+                "hive.sql.dbcp.maxActive" = "1"
+)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@store_sales
+POSTHOOK: query: CREATE EXTERNAL TABLE store_sales
+(
+    ss_sold_date_sk           int                           ,
+    ss_sold_time_sk           int                           ,
+    ss_item_sk                int                           ,
+    ss_customer_sk            int                           ,
+    ss_cdemo_sk               int                           ,
+    ss_hdemo_sk               int                           ,
+    ss_addr_sk                int                           ,
+    ss_store_sk               int                           ,
+    ss_promo_sk               int                           ,
+    ss_ticket_number          int                           ,
+    ss_quantity               int                           ,
+    ss_wholesale_cost         decimal(7,2)                  ,
+    ss_list_price             decimal(7,2)                  ,
+    ss_sales_price            decimal(7,2)                  ,
+    ss_ext_discount_amt       decimal(7,2)                  ,
+    ss_ext_sales_price        decimal(7,2)                  ,
+    ss_ext_wholesale_cost     decimal(7,2)                  ,
+    ss_ext_list_price         decimal(7,2)                  ,
+    ss_ext_tax                decimal(7,2)                  ,
+    ss_coupon_amt             decimal(7,2)                  ,
+    ss_net_paid               decimal(7,2)                  ,
+    ss_net_paid_inc_tax       decimal(7,2)                  ,
+    ss_net_profit             decimal(7,2)
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+#### A masked pattern was here ####
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "STORE_SALES",
+                "hive.sql.dbcp.maxActive" = "1"
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@store_sales
+PREHOOK: query: CREATE EXTERNAL TABLE store_returns
+(
+    sr_returned_date_sk       int                           ,
+    sr_return_time_sk         int                           ,
+    sr_item_sk                int                           ,
+    sr_customer_sk            int                           ,
+    sr_cdemo_sk               int                           ,
+    sr_hdemo_sk               int                           ,
+    sr_addr_sk                int                           ,
+    sr_store_sk               int                           ,
+    sr_reason_sk              int                           ,
+    sr_ticket_number          int                           ,
+    sr_return_quantity        int                           ,
+    sr_return_amt             decimal(7,2)                  ,
+    sr_return_tax             decimal(7,2)                  ,
+    sr_return_amt_inc_tax     decimal(7,2)                  ,
+    sr_fee                    decimal(7,2)                  ,
+    sr_return_ship_cost       decimal(7,2)                  ,
+    sr_refunded_cash          decimal(7,2)                  ,
+    sr_reversed_charge        decimal(7,2)                  ,
+    sr_store_credit           decimal(7,2)                  ,
+    sr_net_loss               decimal(7,2)
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+#### A masked pattern was here ####
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "STORE_RETURNS",
+                "hive.sql.dbcp.maxActive" = "1"
+)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@store_returns
+POSTHOOK: query: CREATE EXTERNAL TABLE store_returns
+(
+    sr_returned_date_sk       int                           ,
+    sr_return_time_sk         int                           ,
+    sr_item_sk                int                           ,
+    sr_customer_sk            int                           ,
+    sr_cdemo_sk               int                           ,
+    sr_hdemo_sk               int                           ,
+    sr_addr_sk                int                           ,
+    sr_store_sk               int                           ,
+    sr_reason_sk              int                           ,
+    sr_ticket_number          int                           ,
+    sr_return_quantity        int                           ,
+    sr_return_amt             decimal(7,2)                  ,
+    sr_return_tax             decimal(7,2)                  ,
+    sr_return_amt_inc_tax     decimal(7,2)                  ,
+    sr_fee                    decimal(7,2)                  ,
+    sr_return_ship_cost       decimal(7,2)                  ,
+    sr_refunded_cash          decimal(7,2)                  ,
+    sr_reversed_charge        decimal(7,2)                  ,
+    sr_store_credit           decimal(7,2)                  ,
+    sr_net_loss               decimal(7,2)
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+#### A masked pattern was here ####
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "STORE_RETURNS",
+                "hive.sql.dbcp.maxActive" = "1"
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@store_returns
+PREHOOK: query: CREATE EXTERNAL TABLE web_sales
+(
+    ws_sold_date_sk           int                           ,
+    ws_sold_time_sk           int                           ,
+    ws_ship_date_sk           int                           ,
+    ws_item_sk                int                           ,
+    ws_bill_customer_sk       int                           ,
+    ws_bill_cdemo_sk          int                           ,
+    ws_bill_hdemo_sk          int                           ,
+    ws_bill_addr_sk           int                           ,
+    ws_ship_customer_sk       int                           ,
+    ws_ship_cdemo_sk          int                           ,
+    ws_ship_hdemo_sk          int                           ,
+    ws_ship_addr_sk           int                           ,
+    ws_web_page_sk            int                           ,
+    ws_web_site_sk            int                           ,
+    ws_ship_mode_sk           int                           ,
+    ws_warehouse_sk           int                           ,
+    ws_promo_sk               int                           ,
+    ws_order_number           int                           ,
+    ws_quantity               int                           ,
+    ws_wholesale_cost         decimal(7,2)                  ,
+    ws_list_price             decimal(7,2)                  ,
+    ws_sales_price            decimal(7,2)                  ,
+    ws_ext_discount_amt       decimal(7,2)                  ,
+    ws_ext_sales_price        decimal(7,2)                  ,
+    ws_ext_wholesale_cost     decimal(7,2)                  ,
+    ws_ext_list_price         decimal(7,2)                  ,
+    ws_ext_tax                decimal(7,2)                  ,
+    ws_coupon_amt             decimal(7,2)                  ,
+    ws_ext_ship_cost          decimal(7,2)                  ,
+    ws_net_paid               decimal(7,2)                  ,
+    ws_net_paid_inc_tax       decimal(7,2)                  ,
+    ws_net_paid_inc_ship      decimal(7,2)                  ,
+    ws_net_paid_inc_ship_tax  decimal(7,2)                  ,
+    ws_net_profit             decimal(7,2)
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+#### A masked pattern was here ####
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "WEB_SALES",
+                "hive.sql.dbcp.maxActive" = "1"
+)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@web_sales
+POSTHOOK: query: CREATE EXTERNAL TABLE web_sales
+(
+    ws_sold_date_sk           int                           ,
+    ws_sold_time_sk           int                           ,
+    ws_ship_date_sk           int                           ,
+    ws_item_sk                int                           ,
+    ws_bill_customer_sk       int                           ,
+    ws_bill_cdemo_sk          int                           ,
+    ws_bill_hdemo_sk          int                           ,
+    ws_bill_addr_sk           int                           ,
+    ws_ship_customer_sk       int                           ,
+    ws_ship_cdemo_sk          int                           ,
+    ws_ship_hdemo_sk          int                           ,
+    ws_ship_addr_sk           int                           ,
+    ws_web_page_sk            int                           ,
+    ws_web_site_sk            int                           ,
+    ws_ship_mode_sk           int                           ,
+    ws_warehouse_sk           int                           ,
+    ws_promo_sk               int                           ,
+    ws_order_number           int                           ,
+    ws_quantity               int                           ,
+    ws_wholesale_cost         decimal(7,2)                  ,
+    ws_list_price             decimal(7,2)                  ,
+    ws_sales_price            decimal(7,2)                  ,
+    ws_ext_discount_amt       decimal(7,2)                  ,
+    ws_ext_sales_price        decimal(7,2)                  ,
+    ws_ext_wholesale_cost     decimal(7,2)                  ,
+    ws_ext_list_price         decimal(7,2)                  ,
+    ws_ext_tax                decimal(7,2)                  ,
+    ws_coupon_amt             decimal(7,2)                  ,
+    ws_ext_ship_cost          decimal(7,2)                  ,
+    ws_net_paid               decimal(7,2)                  ,
+    ws_net_paid_inc_tax       decimal(7,2)                  ,
+    ws_net_paid_inc_ship      decimal(7,2)                  ,
+    ws_net_paid_inc_ship_tax  decimal(7,2)                  ,
+    ws_net_profit             decimal(7,2)
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+#### A masked pattern was here ####
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "WEB_SALES",
+                "hive.sql.dbcp.maxActive" = "1"
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@web_sales
+PREHOOK: query: CREATE EXTERNAL TABLE web_returns
+(
+    wr_returned_date_sk       int                           ,
+    wr_returned_time_sk       int                           ,
+    wr_item_sk                int                           ,
+    wr_refunded_customer_sk   int                           ,
+    wr_refunded_cdemo_sk      int                           ,
+    wr_refunded_hdemo_sk      int                           ,
+    wr_refunded_addr_sk       int                           ,
+    wr_returning_customer_sk  int                           ,
+    wr_returning_cdemo_sk     int                           ,
+    wr_returning_hdemo_sk     int                           ,
+    wr_returning_addr_sk      int                           ,
+    wr_web_page_sk            int                           ,
+    wr_reason_sk              int                           ,
+    wr_order_number           int                           ,
+    wr_return_quantity        int                           ,
+    wr_return_amt             decimal(7,2)                  ,
+    wr_return_tax             decimal(7,2)                  ,
+    wr_return_amt_inc_tax     decimal(7,2)                  ,
+    wr_fee                    decimal(7,2)                  ,
+    wr_return_ship_cost       decimal(7,2)                  ,
+    wr_refunded_cash          decimal(7,2)                  ,
+    wr_reversed_charge        decimal(7,2)                  ,
+    wr_account_credit         decimal(7,2)                  ,
+    wr_net_loss               decimal(7,2)
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+#### A masked pattern was here ####
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "WEB_RETURNS",
+                "hive.sql.dbcp.maxActive" = "1"
+)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@web_returns
+POSTHOOK: query: CREATE EXTERNAL TABLE web_returns
+(
+    wr_returned_date_sk       int                           ,
+    wr_returned_time_sk       int                           ,
+    wr_item_sk                int                           ,
+    wr_refunded_customer_sk   int                           ,
+    wr_refunded_cdemo_sk      int                           ,
+    wr_refunded_hdemo_sk      int                           ,
+    wr_refunded_addr_sk       int                           ,
+    wr_returning_customer_sk  int                           ,
+    wr_returning_cdemo_sk     int                           ,
+    wr_returning_hdemo_sk     int                           ,
+    wr_returning_addr_sk      int                           ,
+    wr_web_page_sk            int                           ,
+    wr_reason_sk              int                           ,
+    wr_order_number           int                           ,
+    wr_return_quantity        int                           ,
+    wr_return_amt             decimal(7,2)                  ,
+    wr_return_tax             decimal(7,2)                  ,
+    wr_return_amt_inc_tax     decimal(7,2)                  ,
+    wr_fee                    decimal(7,2)                  ,
+    wr_return_ship_cost       decimal(7,2)                  ,
+    wr_refunded_cash          decimal(7,2)                  ,
+    wr_reversed_charge        decimal(7,2)                  ,
+    wr_account_credit         decimal(7,2)                  ,
+    wr_net_loss               decimal(7,2)
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+#### A masked pattern was here ####
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "WEB_RETURNS",
+                "hive.sql.dbcp.maxActive" = "1"
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@web_returns
+PREHOOK: query: CREATE EXTERNAL TABLE customer
+(
+    c_customer_sk             int                           ,
+    c_customer_id             char(16)                      ,
+    c_current_cdemo_sk        int                           ,
+    c_current_hdemo_sk        int                           ,
+    c_current_addr_sk         int                           ,
+    c_first_shipto_date_sk    int                           ,
+    c_first_sales_date_sk     int                           ,
+    c_salutation              char(10)                      ,
+    c_first_name              char(20)                      ,
+    c_last_name               char(30)                      ,
+    c_preferred_cust_flag     char(1)                       ,
+    c_birth_day               int                           ,
+    c_birth_month             int                           ,
+    c_birth_year              int                           ,
+    c_birth_country           varchar(20)                   ,
+    c_login                   char(13)                      ,
+    c_email_address           char(50)                      ,
+    c_last_review_date        char(10)
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+#### A masked pattern was here ####
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "CUSTOMER",
+                "hive.sql.dbcp.maxActive" = "1"
+)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@customer
+POSTHOOK: query: CREATE EXTERNAL TABLE customer
+(
+    c_customer_sk             int                           ,
+    c_customer_id             char(16)                      ,
+    c_current_cdemo_sk        int                           ,
+    c_current_hdemo_sk        int                           ,
+    c_current_addr_sk         int                           ,
+    c_first_shipto_date_sk    int                           ,
+    c_first_sales_date_sk     int                           ,
+    c_salutation              char(10)                      ,
+    c_first_name              char(20)                      ,
+    c_last_name               char(30)                      ,
+    c_preferred_cust_flag     char(1)                       ,
+    c_birth_day               int                           ,
+    c_birth_month             int                           ,
+    c_birth_year              int                           ,
+    c_birth_country           varchar(20)                   ,
+    c_login                   char(13)                      ,
+    c_email_address           char(50)                      ,
+    c_last_review_date        char(10)
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+#### A masked pattern was here ####
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "CUSTOMER",
+                "hive.sql.dbcp.maxActive" = "1"
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@customer
+PREHOOK: query: CREATE EXTERNAL TABLE customer_address
+(
+    ca_address_sk             int                           ,
+    ca_address_id             char(16)                      ,
+    ca_street_number          char(10)                      ,
+    ca_street_name            varchar(60)                   ,
+    ca_street_type            char(15)                      ,
+    ca_suite_number           char(10)                      ,
+    ca_city                   varchar(60)                   ,
+    ca_county                 varchar(30)                   ,
+    ca_state                  char(2)                       ,
+    ca_zip                    char(10)                      ,
+    ca_country                varchar(20)                   ,
+    ca_gmt_offset             decimal(5,2)                  ,
+    ca_location_type          char(20)
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+#### A masked pattern was here ####
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "CUSTOMER_ADDRESS",
+                "hive.sql.dbcp.maxActive" = "1"
+)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@customer_address
+POSTHOOK: query: CREATE EXTERNAL TABLE customer_address
+(
+    ca_address_sk             int                           ,
+    ca_address_id             char(16)                      ,
+    ca_street_number          char(10)                      ,
+    ca_street_name            varchar(60)                   ,
+    ca_street_type            char(15)                      ,
+    ca_suite_number           char(10)                      ,
+    ca_city                   varchar(60)                   ,
+    ca_county                 varchar(30)                   ,
+    ca_state                  char(2)                       ,
+    ca_zip                    char(10)                      ,
+    ca_country                varchar(20)                   ,
+    ca_gmt_offset             decimal(5,2)                  ,
+    ca_location_type          char(20)
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+#### A masked pattern was here ####
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "CUSTOMER_ADDRESS",
+                "hive.sql.dbcp.maxActive" = "1"
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@customer_address
+PREHOOK: query: CREATE EXTERNAL TABLE customer_demographics
+(
+    cd_demo_sk                int                           ,
+    cd_gender                 char(1)                       ,
+    cd_marital_status         char(1)                       ,
+    cd_education_status       char(20)                      ,
+    cd_purchase_estimate      int                           ,
+    cd_credit_rating          char(10)                      ,
+    cd_dep_count              int                           ,
+    cd_dep_employed_count     int                           ,
+    cd_dep_college_count      int
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+#### A masked pattern was here ####
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "CUSTOMER_DEMOGRAPHICS",
+                "hive.sql.dbcp.maxActive" = "1"
+)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@customer_demographics
+POSTHOOK: query: CREATE EXTERNAL TABLE customer_demographics
+(
+    cd_demo_sk                int                           ,
+    cd_gender                 char(1)                       ,
+    cd_marital_status         char(1)                       ,
+    cd_education_status       char(20)                      ,
+    cd_purchase_estimate      int                           ,
+    cd_credit_rating          char(10)                      ,
+    cd_dep_count              int                           ,
+    cd_dep_employed_count     int                           ,
+    cd_dep_college_count      int
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+#### A masked pattern was here ####
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "CUSTOMER_DEMOGRAPHICS",
+                "hive.sql.dbcp.maxActive" = "1"
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@customer_demographics
+PREHOOK: query: CREATE EXTERNAL TABLE inventory
+(
+    inv_date_sk               int                           ,
+    inv_item_sk               int                           ,
+    inv_warehouse_sk          int                           ,
+    inv_quantity_on_hand      int
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+#### A masked pattern was here ####
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "INVENTORY",
+                "hive.sql.dbcp.maxActive" = "1"
+)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@inventory
+POSTHOOK: query: CREATE EXTERNAL TABLE inventory
+(
+    inv_date_sk               int                           ,
+    inv_item_sk               int                           ,
+    inv_warehouse_sk          int                           ,
+    inv_quantity_on_hand      int
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+#### A masked pattern was here ####
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "INVENTORY",
+                "hive.sql.dbcp.maxActive" = "1"
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@inventory
+PREHOOK: query: CREATE EXTERNAL TABLE item
+(
+    i_item_sk                 int                           ,
+    i_item_id                 char(16)                      ,
+    i_rec_start_date          date                          ,
+    i_rec_end_date            date                          ,
+    i_item_desc               varchar(200)                  ,
+    i_current_price           decimal(7,2)                  ,
+    i_wholesale_cost          decimal(7,2)                  ,
+    i_brand_id                int                           ,
+    i_brand                   char(50)                      ,
+    i_class_id                int                           ,
+    i_class                   char(50)                      ,
+    i_category_id             int                           ,
+    i_category                char(50)                      ,
+    i_manufact_id             int                           ,
+    i_manufact                char(50)                      ,
+    i_size                    char(20)                      ,
+    i_formulation             char(20)                      ,
+    i_color                   char(20)                      ,
+    i_units                   char(10)                      ,
+    i_container               char(10)                      ,
+    i_manager_id              int                           ,
+    i_product_name            char(50)
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+#### A masked pattern was here ####
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "ITEM",
+                "hive.sql.dbcp.maxActive" = "1"
+)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@item
+POSTHOOK: query: CREATE EXTERNAL TABLE item
+(
+    i_item_sk                 int                           ,
+    i_item_id                 char(16)                      ,
+    i_rec_start_date          date                          ,
+    i_rec_end_date            date                          ,
+    i_item_desc               varchar(200)                  ,
+    i_current_price           decimal(7,2)                  ,
+    i_wholesale_cost          decimal(7,2)                  ,
+    i_brand_id                int                           ,
+    i_brand                   char(50)                      ,
+    i_class_id                int                           ,
+    i_class                   char(50)                      ,
+    i_category_id             int                           ,
+    i_category                char(50)                      ,
+    i_manufact_id             int                           ,
+    i_manufact                char(50)                      ,
+    i_size                    char(20)                      ,
+    i_formulation             char(20)                      ,
+    i_color                   char(20)                      ,
+    i_units                   char(10)                      ,
+    i_container               char(10)                      ,
+    i_manager_id              int                           ,
+    i_product_name            char(50)
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+#### A masked pattern was here ####
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "ITEM",
+                "hive.sql.dbcp.maxActive" = "1"
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@item
+PREHOOK: query: CREATE EXTERNAL TABLE warehouse
+(
+    w_warehouse_sk            int                           ,
+    w_warehouse_id            char(16)                      ,
+    w_warehouse_name          varchar(20)                   ,
+    w_warehouse_sq_ft         int                           ,
+    w_street_number           char(10)                      ,
+    w_street_name             varchar(60)                   ,
+    w_street_type             char(15)                      ,
+    w_suite_number            char(10)                      ,
+    w_city                    varchar(60)                   ,
+    w_county                  varchar(30)                   ,
+    w_state                   char(2)                       ,
+    w_zip                     char(10)                      ,
+    w_country                 varchar(20)                   ,
+    w_gmt_offset              decimal(5,2)
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+#### A masked pattern was here ####
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "WAREHOUSE",
+                "hive.sql.dbcp.maxActive" = "1"
+)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@warehouse
+POSTHOOK: query: CREATE EXTERNAL TABLE warehouse
+(
+    w_warehouse_sk            int                           ,
+    w_warehouse_id            char(16)                      ,
+    w_warehouse_name          varchar(20)                   ,
+    w_warehouse_sq_ft         int                           ,
+    w_street_number           char(10)                      ,
+    w_street_name             varchar(60)                   ,
+    w_street_type             char(15)                      ,
+    w_suite_number            char(10)                      ,
+    w_city                    varchar(60)                   ,
+    w_county                  varchar(30)                   ,
+    w_state                   char(2)                       ,
+    w_zip                     char(10)                      ,
+    w_country                 varchar(20)                   ,
+    w_gmt_offset              decimal(5,2)
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+#### A masked pattern was here ####
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "WAREHOUSE",
+                "hive.sql.dbcp.maxActive" = "1"
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@warehouse
+PREHOOK: query: CREATE EXTERNAL TABLE date_dim
+(
+    d_date_sk                 int                           ,
+    d_date_id                 char(16)                      ,
+    d_date                    date                          ,
+    d_month_seq               int                           ,
+    d_week_seq                int                           ,
+    d_quarter_seq             int                           ,
+    d_year                    int                           ,
+    d_dow                     int                           ,
+    d_moy                     int                           ,
+    d_dom                     int                           ,
+    d_qoy                     int                           ,
+    d_fy_year                 int                           ,
+    d_fy_quarter_seq          int                           ,
+    d_fy_week_seq             int                           ,
+    d_day_name                char(9)                       ,
+    d_quarter_name            char(6)                       ,
+    d_holiday                 char(1)                       ,
+    d_weekend                 char(1)                       ,
+    d_following_holiday       char(1)                       ,
+    d_first_dom               int                           ,
+    d_last_dom                int                           ,
+    d_same_day_ly             int                           ,
+    d_same_day_lq             int                           ,
+    d_current_day             char(1)                       ,
+    d_current_week            char(1)                       ,
+    d_current_month           char(1)                       ,
+    d_current_quarter         char(1)                       ,
+    d_current_year            char(1)
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+#### A masked pattern was here ####
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "DATE_DIM",
+                "hive.sql.dbcp.maxActive" = "1"
+)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@date_dim
+POSTHOOK: query: CREATE EXTERNAL TABLE date_dim
+(
+    d_date_sk                 int                           ,
+    d_date_id                 char(16)                      ,
+    d_date                    date                          ,
+    d_month_seq               int                           ,
+    d_week_seq                int                           ,
+    d_quarter_seq             int                           ,
+    d_year                    int                           ,
+    d_dow                     int                           ,
+    d_moy                     int                           ,
+    d_dom                     int                           ,
+    d_qoy                     int                           ,
+    d_fy_year                 int                           ,
+    d_fy_quarter_seq          int                           ,
+    d_fy_week_seq             int                           ,
+    d_day_name                char(9)                       ,
+    d_quarter_name            char(6)                       ,
+    d_holiday                 char(1)                       ,
+    d_weekend                 char(1)                       ,
+    d_following_holiday       char(1)                       ,
+    d_first_dom               int                           ,
+    d_last_dom                int                           ,
+    d_same_day_ly             int                           ,
+    d_same_day_lq             int                           ,
+    d_current_day             char(1)                       ,
+    d_current_week            char(1)                       ,
+    d_current_month           char(1)                       ,
+    d_current_quarter         char(1)                       ,
+    d_current_year            char(1)
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+#### A masked pattern was here ####
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "DATE_DIM",
+                "hive.sql.dbcp.maxActive" = "1"
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@date_dim
+PREHOOK: query: CREATE EXTERNAL TABLE store
+(
+    s_store_sk                int                           ,
+    s_store_id                char(16)                      ,
+    s_rec_start_date          date                          ,
+    s_rec_end_date            date                          ,
+    s_closed_date_sk          int                           ,
+    s_store_name              varchar(50)                   ,
+    s_number_employees        int                           ,
+    s_floor_space             int                           ,
+    s_hours                   char(20)                      ,
+    s_manager                 varchar(40)                   ,
+    s_market_id               int                           ,
+    s_geography_class         varchar(100)                  ,
+    s_market_desc             varchar(100)                  ,
+    s_market_manager          varchar(40)                   ,
+    s_division_id             int                           ,
+    s_division_name           varchar(50)                   ,
+    s_company_id              int                           ,
+    s_company_name            varchar(50)                   ,
+    s_street_number           varchar(10)                   ,
+    s_street_name             varchar(60)                   ,
+    s_street_type             char(15)                      ,
+    s_suite_number            char(10)                      ,
+    s_city                    varchar(60)                   ,
+    s_county                  varchar(30)                   ,
+    s_state                   char(2)                       ,
+    s_zip                     char(10)                      ,
+    s_country                 varchar(20)                   ,
+    s_gmt_offset              decimal(5,2)                  ,
+    s_tax_precentage          decimal(5,2)                  
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+#### A masked pattern was here ####
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "STORE",
+                "hive.sql.dbcp.maxActive" = "1"
+)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@store
+POSTHOOK: query: CREATE EXTERNAL TABLE store
+(
+    s_store_sk                int                           ,
+    s_store_id                char(16)                      ,
+    s_rec_start_date          date                          ,
+    s_rec_end_date            date                          ,
+    s_closed_date_sk          int                           ,
+    s_store_name              varchar(50)                   ,
+    s_number_employees        int                           ,
+    s_floor_space             int                           ,
+    s_hours                   char(20)                      ,
+    s_manager                 varchar(40)                   ,
+    s_market_id               int                           ,
+    s_geography_class         varchar(100)                  ,
+    s_market_desc             varchar(100)                  ,
+    s_market_manager          varchar(40)                   ,
+    s_division_id             int                           ,
+    s_division_name           varchar(50)                   ,
+    s_company_id              int                           ,
+    s_company_name            varchar(50)                   ,
+    s_street_number           varchar(10)                   ,
+    s_street_name             varchar(60)                   ,
+    s_street_type             char(15)                      ,
+    s_suite_number            char(10)                      ,
+    s_city                    varchar(60)                   ,
+    s_county                  varchar(30)                   ,
+    s_state                   char(2)                       ,
+    s_zip                     char(10)                      ,
+    s_country                 varchar(20)                   ,
+    s_gmt_offset              decimal(5,2)                  ,
+    s_tax_precentage          decimal(5,2)                  
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+#### A masked pattern was here ####
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "STORE",
+                "hive.sql.dbcp.maxActive" = "1"
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@store
+PREHOOK: query: CREATE EXTERNAL TABLE household_demographics
+(
+    hd_demo_sk                int                           ,
+    hd_income_band_sk         int                           ,
+    hd_buy_potential          char(15)                      ,
+    hd_dep_count              int                           ,
+    hd_vehicle_count          int                           
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+#### A masked pattern was here ####
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "HOUSEHOLD_DEMOGRAPHICS",
+                "hive.sql.dbcp.maxActive" = "1"
+)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@household_demographics
+POSTHOOK: query: CREATE EXTERNAL TABLE household_demographics
+(
+    hd_demo_sk                int                           ,
+    hd_income_band_sk         int                           ,
+    hd_buy_potential          char(15)                      ,
+    hd_dep_count              int                           ,
+    hd_vehicle_count          int                           
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+#### A masked pattern was here ####
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "HOUSEHOLD_DEMOGRAPHICS",
+                "hive.sql.dbcp.maxActive" = "1"
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@household_demographics
+PREHOOK: query: CREATE EXTERNAL TABLE time_dim
+(
+    t_time_sk                 int                           ,
+    t_time_id                 char(16)                      ,
+    t_time                    int                           ,
+    t_hour                    int                           ,
+    t_minute                  int                           ,
+    t_second                  int                           ,
+    t_am_pm                   char(2)                       ,
+    t_shift                   char(20)                      ,
+    t_sub_shift               char(20)                      ,
+    t_meal_time               char(20)                      
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+#### A masked pattern was here ####
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "TIME_DIM",
+                "hive.sql.dbcp.maxActive" = "1"
+)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@time_dim
+POSTHOOK: query: CREATE EXTERNAL TABLE time_dim
+(
+    t_time_sk                 int                           ,
+    t_time_id                 char(16)                      ,
+    t_time                    int                           ,
+    t_hour                    int                           ,
+    t_minute                  int                           ,
+    t_second                  int                           ,
+    t_am_pm                   char(2)                       ,
+    t_shift                   char(20)                      ,
+    t_sub_shift               char(20)                      ,
+    t_meal_time               char(20)                      
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+#### A masked pattern was here ####
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "TIME_DIM",
+                "hive.sql.dbcp.maxActive" = "1"
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@time_dim
+PREHOOK: query: CREATE EXTERNAL TABLE reason
+(
+    r_reason_sk               int                           ,
+    r_reason_id               char(16)                      ,
+    r_reason_desc             char(100)
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+#### A masked pattern was here ####
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "REASON",
+                "hive.sql.dbcp.maxActive" = "1"
+)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@reason
+POSTHOOK: query: CREATE EXTERNAL TABLE reason
+(
+    r_reason_sk               int                           ,
+    r_reason_id               char(16)                      ,
+    r_reason_desc             char(100)
+)
+STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler'
+TBLPROPERTIES (
+                "hive.sql.database.type" = "DERBY",
+                "hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver",
+#### A masked pattern was here ####
+                "hive.sql.dbcp.username" = "user1",
+                "hive.sql.dbcp.password" = "passwd1",
+                "hive.sql.table" = "REASON",
+                "hive.sql.dbcp.maxActive" = "1"
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@reason
+PREHOOK: query: explain
+select inv1.w_warehouse_sk, inv1.i_item_sk, inv1.d_moy, inv1.mean, inv1.cov,
+       inv2.w_warehouse_sk, inv2.i_item_sk, inv2.d_moy, inv2.mean, inv2.cov
+from (select w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy,
+             stdev, mean, case mean when 0.0
+                                    then null else stdev/mean end cov
+      from (select w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy,
+                   sum(inv_quantity_on_hand) as stdev,
+                   avg(inv_quantity_on_hand) as mean
+            from inventory
+            join item on inventory.inv_item_sk = item.i_item_sk
+            join warehouse on inventory.inv_warehouse_sk = warehouse.w_warehouse_sk
+            join date_dim on inventory.inv_date_sk = date_dim.d_date_sk
+            where d_year = 2001
+            group by w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy) foo
+      where case mean when 0.0
+                      then 0.0 else stdev/mean end > 1) inv1
+join (select w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy,
+             stdev, mean, case mean when 0.0
+                                    then null else stdev/mean end cov
+      from (select w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy,
+                   sum(inv_quantity_on_hand) as stdev,
+                   avg(inv_quantity_on_hand) as mean
+            from inventory
+            join item on inventory.inv_item_sk = item.i_item_sk
+            join warehouse on inventory.inv_warehouse_sk = warehouse.w_warehouse_sk
+            join date_dim on inventory.inv_date_sk = date_dim.d_date_sk
+            where d_year = 2001
+            group by w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy) foo
+      where case mean when 0.0
+                      then 0.0 else stdev/mean end > 1) inv2
+  on inv1.i_item_sk = inv2.i_item_sk
+     and inv1.w_warehouse_sk = inv2.w_warehouse_sk
+where inv1.d_moy = 1 and inv2.d_moy = 1+1
+order by inv1.w_warehouse_sk, inv1.i_item_sk, inv1.d_moy, inv1.mean, inv1.cov,
+         inv2.d_moy, inv2.mean, inv2.cov
+PREHOOK: type: QUERY
+PREHOOK: Input: default@date_dim
+PREHOOK: Input: default@inventory
+PREHOOK: Input: default@item
+PREHOOK: Input: default@warehouse
+#### A masked pattern was here ####
+POSTHOOK: query: explain
+select inv1.w_warehouse_sk, inv1.i_item_sk, inv1.d_moy, inv1.mean, inv1.cov,
+       inv2.w_warehouse_sk, inv2.i_item_sk, inv2.d_moy, inv2.mean, inv2.cov
+from (select w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy,
+             stdev, mean, case mean when 0.0
+                                    then null else stdev/mean end cov
+      from (select w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy,
+                   sum(inv_quantity_on_hand) as stdev,
+                   avg(inv_quantity_on_hand) as mean
+            from inventory
+            join item on inventory.inv_item_sk = item.i_item_sk
+            join warehouse on inventory.inv_warehouse_sk = warehouse.w_warehouse_sk
+            join date_dim on inventory.inv_date_sk = date_dim.d_date_sk
+            where d_year = 2001
+            group by w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy) foo
+      where case mean when 0.0
+                      then 0.0 else stdev/mean end > 1) inv1
+join (select w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy,
+             stdev, mean, case mean when 0.0
+                                    then null else stdev/mean end cov
+      from (select w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy,
+                   sum(inv_quantity_on_hand) as stdev,
+                   avg(inv_quantity_on_hand) as mean
+            from inventory
+            join item on inventory.inv_item_sk = item.i_item_sk
+            join warehouse on inventory.inv_warehouse_sk = warehouse.w_warehouse_sk
+            join date_dim on inventory.inv_date_sk = date_dim.d_date_sk
+            where d_year = 2001
+            group by w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy) foo
+      where case mean when 0.0
+                      then 0.0 else stdev/mean end > 1) inv2
+  on inv1.i_item_sk = inv2.i_item_sk
+     and inv1.w_warehouse_sk = inv2.w_warehouse_sk
+where inv1.d_moy = 1 and inv2.d_moy = 1+1
+order by inv1.w_warehouse_sk, inv1.i_item_sk, inv1.d_moy, inv1.mean, inv1.cov,
+         inv2.d_moy, inv2.mean, inv2.cov
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@date_dim
+POSTHOOK: Input: default@inventory
+POSTHOOK: Input: default@item
+POSTHOOK: Input: default@warehouse
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: inventory
+          properties:
+            hive.sql.query SELECT "t23"."w_warehouse_sk", "t23"."i_item_sk", CAST(1 AS INTEGER) AS "d_moy", "t23"."mean", "t23"."cov", "t23"."w_warehouse_sk0" AS "w_warehouse_sk1", "t23"."i_item_sk0" AS "i_item_sk1", CAST(2 AS INTEGER) AS "d_moy1", "t23"."mean0" AS "mean1", "t23"."cov0" AS "cov1"
+FROM (SELECT "t10"."w_warehouse_sk", "t10"."i_item_sk", "t10"."mean", "t10"."cov", "t22"."w_warehouse_sk" AS "w_warehouse_sk0", "t22"."i_item_sk" AS "i_item_sk0", "t22"."mean" AS "mean0", "t22"."cov" AS "cov0"
+FROM (SELECT "w_warehouse_sk", "i_item_sk", CAST("$f3" AS DOUBLE) / "$f4" AS "mean", CASE WHEN CAST("$f3" AS DOUBLE) / "$f4" = 0 THEN NULL ELSE CAST("$f3" AS DOUBLE) / (CAST("$f3" AS DOUBLE) / "$f4") END AS "cov"
+FROM (SELECT "t4"."w_warehouse_name", "t4"."w_warehouse_sk", "t2"."i_item_sk", SUM("t0"."inv_quantity_on_hand") AS "$f3", COUNT("t0"."inv_quantity_on_hand") AS "$f4"
+FROM (SELECT "inv_date_sk", "inv_item_sk", "inv_warehouse_sk", "inv_quantity_on_hand"
+FROM "INVENTORY"
+WHERE "inv_item_sk" IS NOT NULL AND "inv_warehouse_sk" IS NOT NULL AND "inv_date_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "i_item_sk"
+FROM "ITEM"
+WHERE "i_item_sk" IS NOT NULL) AS "t2" ON "t0"."inv_item_sk" = "t2"."i_item_sk"
+INNER JOIN (SELECT "w_warehouse_sk", "w_warehouse_name"
+FROM "WAREHOUSE"
+WHERE "w_warehouse_sk" IS NOT NULL) AS "t4" ON "t0"."inv_warehouse_sk" = "t4"."w_warehouse_sk"
+INNER JOIN (SELECT "d_date_sk"
+FROM "DATE_DIM"
+WHERE "d_year" = 2001 AND "d_moy" = 1 AND "d_date_sk" IS NOT NULL) AS "t6" ON "t0"."inv_date_sk" = "t6"."d_date_sk"
+GROUP BY "t2"."i_item_sk", "t4"."w_warehouse_sk", "t4"."w_warehouse_name") AS "t8"
+WHERE CASE WHEN CAST("t8"."$f3" AS DOUBLE) / "t8"."$f4" = 0 THEN FALSE ELSE CAST("t8"."$f3" AS DOUBLE) / (CAST("t8"."$f3" AS DOUBLE) / "t8"."$f4") > 1 END) AS "t10"
+INNER JOIN (SELECT "w_warehouse_sk", "i_item_sk", CAST("$f3" AS DOUBLE) / "$f4" AS "mean", CASE WHEN CAST("$f3" AS DOUBLE) / "$f4" = 0 THEN NULL ELSE CAST("$f3" AS DOUBLE) / (CAST("$f3" AS DOUBLE) / "$f4") END AS "cov"
+FROM (SELECT "t16"."w_warehouse_name", "t16"."w_warehouse_sk", "t14"."i_item_sk", SUM("t12"."inv_quantity_on_hand") AS "$f3", COUNT("t12"."inv_quantity_on_hand") AS "$f4"
+FROM (SELECT "inv_date_sk", "inv_item_sk", "inv_warehouse_sk", "inv_quantity_on_hand"
+FROM "INVENTORY"
+WHERE "inv_item_sk" IS NOT NULL AND "inv_warehouse_sk" IS NOT NULL AND "inv_date_sk" IS NOT NULL) AS "t12"
+INNER JOIN (SELECT "i_item_sk"
+FROM "ITEM"
+WHERE "i_item_sk" IS NOT NULL) AS "t14" ON "t12"."inv_item_sk" = "t14"."i_item_sk"
+INNER JOIN (SELECT "w_warehouse_sk", "w_warehouse_name"
+FROM "WAREHOUSE"
+WHERE "w_warehouse_sk" IS NOT NULL) AS "t16" ON "t12"."inv_warehouse_sk" = "t16"."w_warehouse_sk"
+INNER JOIN (SELECT "d_date_sk"
+FROM "DATE_DIM"
+WHERE "d_year" = 2001 AND "d_moy" = 2 AND "d_date_sk" IS NOT NULL) AS "t18" ON "t12"."inv_date_sk" = "t18"."d_date_sk"
+GROUP BY "t14"."i_item_sk", "t16"."w_warehouse_sk", "t16"."w_warehouse_name") AS "t20"
+WHERE CASE WHEN CAST("t20"."$f3" AS DOUBLE) / "t20"."$f4" = 0 THEN FALSE ELSE CAST("t20"."$f3" AS DOUBLE) / (CAST("t20"."$f3" AS DOUBLE) / "t20"."$f4") > 1 END) AS "t22" ON "t10"."i_item_sk" = "t22"."i_item_sk" AND "t10"."w_warehouse_sk" = "t22"."w_warehouse_sk"
+ORDER BY "t10"."w_warehouse_sk", "t10"."i_item_sk", "t10"."mean", "t10"."cov", "t22"."mean", "t22"."cov") AS "t23"
+            hive.sql.query.fieldNames w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk1,i_item_sk1,d_moy1,mean1,cov1
+            hive.sql.query.fieldTypes int,int,int,double,double,int,int,int,double,double
+            hive.sql.query.split false
+          Select Operator
+            expressions: w_warehouse_sk (type: int), i_item_sk (type: int), d_moy (type: int), mean (type: double), cov (type: double), w_warehouse_sk1 (type: int), i_item_sk1 (type: int), d_moy1 (type: int), mean1 (type: double), cov1 (type: double)
+            outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
+            ListSink
+
+PREHOOK: query: select inv1.w_warehouse_sk, inv1.i_item_sk, inv1.d_moy, inv1.mean, inv1.cov,
+       inv2.w_warehouse_sk, inv2.i_item_sk, inv2.d_moy, inv2.mean, inv2.cov
+from (select w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy,
+             stdev, mean, case mean when 0.0
+                                    then null else stdev/mean end cov
+      from (select w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy,
+                   sum(inv_quantity_on_hand) as stdev,
+                   avg(inv_quantity_on_hand) as mean
+            from inventory
+            join item on inventory.inv_item_sk = item.i_item_sk
+            join warehouse on inventory.inv_warehouse_sk = warehouse.w_warehouse_sk
+            join date_dim on inventory.inv_date_sk = date_dim.d_date_sk
+            where d_year = 2001
+            group by w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy) foo
+      where case mean when 0.0
+                      then 0.0 else stdev/mean end > 1) inv1
+join (select w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy,
+             stdev, mean, case mean when 0.0
+                                    then null else stdev/mean end cov
+      from (select w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy,
+                   sum(inv_quantity_on_hand) as stdev,
+                   avg(inv_quantity_on_hand) as mean
+            from inventory
+            join item on inventory.inv_item_sk = item.i_item_sk
+            join warehouse on inventory.inv_warehouse_sk = warehouse.w_warehouse_sk
+            join date_dim on inventory.inv_date_sk = date_dim.d_date_sk
+            where d_year = 2001
+            group by w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy) foo
+      where case mean when 0.0
+                      then 0.0 else stdev/mean end > 1) inv2
+  on inv1.i_item_sk = inv2.i_item_sk
+     and inv1.w_warehouse_sk = inv2.w_warehouse_sk
+where inv1.d_moy = 1 and inv2.d_moy = 1+1
+order by inv1.w_warehouse_sk, inv1.i_item_sk, inv1.d_moy, inv1.mean, inv1.cov,
+         inv2.d_moy, inv2.mean, inv2.cov
+PREHOOK: type: QUERY
+PREHOOK: Input: default@date_dim
+PREHOOK: Input: default@inventory
+PREHOOK: Input: default@item
+PREHOOK: Input: default@warehouse
+#### A masked pattern was here ####
+POSTHOOK: query: select inv1.w_warehouse_sk, inv1.i_item_sk, inv1.d_moy, inv1.mean, inv1.cov,
+       inv2.w_warehouse_sk, inv2.i_item_sk, inv2.d_moy, inv2.mean, inv2.cov
+from (select w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy,
+             stdev, mean, case mean when 0.0
+                                    then null else stdev/mean end cov
+      from (select w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy,
+                   sum(inv_quantity_on_hand) as stdev,
+                   avg(inv_quantity_on_hand) as mean
+            from inventory
+            join item on inventory.inv_item_sk = item.i_item_sk
+            join warehouse on inventory.inv_warehouse_sk = warehouse.w_warehouse_sk
+            join date_dim on inventory.inv_date_sk = date_dim.d_date_sk
+            where d_year = 2001
+            group by w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy) foo
+      where case mean when 0.0
+                      then 0.0 else stdev/mean end > 1) inv1
+join (select w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy,
+             stdev, mean, case mean when 0.0
+                                    then null else stdev/mean end cov
+      from (select w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy,
+                   sum(inv_quantity_on_hand) as stdev,
+                   avg(inv_quantity_on_hand) as mean
+            from inventory
+            join item on inventory.inv_item_sk = item.i_item_sk
+            join warehouse on inventory.inv_warehouse_sk = warehouse.w_warehouse_sk
+            join date_dim on inventory.inv_date_sk = date_dim.d_date_sk
+            where d_year = 2001
+            group by w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy) foo
+      where case mean when 0.0
+                      then 0.0 else stdev/mean end > 1) inv2
+  on inv1.i_item_sk = inv2.i_item_sk
+     and inv1.w_warehouse_sk = inv2.w_warehouse_sk
+where inv1.d_moy = 1 and inv2.d_moy = 1+1
+order by inv1.w_warehouse_sk, inv1.i_item_sk, inv1.d_moy, inv1.mean, inv1.cov,
+         inv2.d_moy, inv2.mean, inv2.cov
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@date_dim
+POSTHOOK: Input: default@inventory
+POSTHOOK: Input: default@item
+POSTHOOK: Input: default@warehouse
+#### A masked pattern was here ####
+PREHOOK: query: explain
+SELECT cd_gender, 
+       cd_marital_status, 
+       cd_education_status, 
+       Count(*) cnt1, 
+       cd_purchase_estimate, 
+       Count(*) cnt2, 
+       cd_credit_rating, 
+       Count(*) cnt3 
+FROM   customer c, 
+       customer_address ca, 
+       customer_demographics 
+WHERE  c.c_current_addr_sk = ca.ca_address_sk 
+       AND ca_state IN ( 'CO', 'IL', 'MN' ) 
+       AND cd_demo_sk = c.c_current_cdemo_sk 
+       AND EXISTS (SELECT * 
+                   FROM   store_sales, 
+                          date_dim 
+                   WHERE  c.c_customer_sk = ss_customer_sk 
+                          AND ss_sold_date_sk = d_date_sk 
+                          AND d_year = 1999 
+                          AND d_moy BETWEEN 1 AND 1 + 2) 
+       AND ( NOT EXISTS (SELECT * 
+                         FROM   web_sales, 
+                                date_dim 
+                         WHERE  c.c_customer_sk = ws_bill_customer_sk 
+                                AND ws_sold_date_sk = d_date_sk 
+                                AND d_year = 1999 
+                                AND d_moy BETWEEN 1 AND 1 + 2) 
+             AND NOT EXISTS (SELECT * 
+                             FROM   catalog_sales, 
+                                    date_dim 
+                             WHERE  c.c_customer_sk = cs_ship_customer_sk 
+                                    AND cs_sold_date_sk = d_date_sk 
+                                    AND d_year = 1999 
+                                    AND d_moy BETWEEN 1 AND 1 + 2) ) 
+GROUP  BY cd_gender, 
+          cd_marital_status, 
+          cd_education_status, 
+          cd_purchase_estimate, 
+          cd_credit_rating 
+ORDER  BY cd_gender, 
+          cd_marital_status, 
+          cd_education_status, 
+          cd_purchase_estimate, 
+          cd_credit_rating 
+LIMIT  100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@catalog_sales
+PREHOOK: Input: default@customer
+PREHOOK: Input: default@customer_address
+PREHOOK: Input: default@customer_demographics
+PREHOOK: Input: default@date_dim
+PREHOOK: Input: default@store_sales
+PREHOOK: Input: default@web_sales
+#### A masked pattern was here ####
+POSTHOOK: query: explain
+SELECT cd_gender, 
+       cd_marital_status, 
+       cd_education_status, 
+       Count(*) cnt1, 
+       cd_purchase_estimate, 
+       Count(*) cnt2, 
+       cd_credit_rating, 
+       Count(*) cnt3 
+FROM   customer c, 
+       customer_address ca, 
+       customer_demographics 
+WHERE  c.c_current_addr_sk = ca.ca_address_sk 
+       AND ca_state IN ( 'CO', 'IL', 'MN' ) 
+       AND cd_demo_sk = c.c_current_cdemo_sk 
+       AND EXISTS (SELECT * 
+                   FROM   store_sales, 
+                          date_dim 
+                   WHERE  c.c_customer_sk = ss_customer_sk 
+                          AND ss_sold_date_sk = d_date_sk 
+                          AND d_year = 1999 
+                          AND d_moy BETWEEN 1 AND 1 + 2) 
+       AND ( NOT EXISTS (SELECT * 
+                         FROM   web_sales, 
+                                date_dim 
+                         WHERE  c.c_customer_sk = ws_bill_customer_sk 
+                                AND ws_sold_date_sk = d_date_sk 
+                                AND d_year = 1999 
+                                AND d_moy BETWEEN 1 AND 1 + 2) 
+             AND NOT EXISTS (SELECT * 
+                             FROM   catalog_sales, 
+                                    date_dim 
+                             WHERE  c.c_customer_sk = cs_ship_customer_sk 
+                                    AND cs_sold_date_sk = d_date_sk 
+                                    AND d_year = 1999 
+                                    AND d_moy BETWEEN 1 AND 1 + 2) ) 
+GROUP  BY cd_gender, 
+          cd_marital_status, 
+          cd_education_status, 
+          cd_purchase_estimate, 
+          cd_credit_rating 
+ORDER  BY cd_gender, 
+          cd_marital_status, 
+          cd_education_status, 
+          cd_purchase_estimate, 
+          cd_credit_rating 
+LIMIT  100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@catalog_sales
+POSTHOOK: Input: default@customer
+POSTHOOK: Input: default@customer_address
+POSTHOOK: Input: default@customer_demographics
+POSTHOOK: Input: default@date_dim
+POSTHOOK: Input: default@store_sales
+POSTHOOK: Input: default@web_sales
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
+        Reducer 3 <- Map 8 (SIMPLE_EDGE), Reducer 2 (ONE_TO_ONE_EDGE)
+        Reducer 4 <- Map 9 (SIMPLE_EDGE), Reducer 3 (ONE_TO_ONE_EDGE)
+        Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
+        Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: c
+                  properties:
+                    hive.sql.query SELECT "t0"."c_customer_sk", "t0"."c_customer_id", "t0"."c_current_cdemo_sk", "t0"."c_current_hdemo_sk", "t0"."c_current_addr_sk", "t0"."c_first_shipto_date_sk", "t0"."c_first_sales_date_sk", "t0"."c_salutation", "t0"."c_first_name", "t0"."c_last_name", "t0"."c_preferred_cust_flag", "t0"."c_birth_day", "t0"."c_birth_month", "t0"."c_birth_year", "t0"."c_birth_country", "t0"."c_login", "t0"."c_email_address", "t0"."c_last_review_date", "t2"."ca_address_sk [...]
+FROM (SELECT "c_customer_sk", "c_customer_id", "c_current_cdemo_sk", "c_current_hdemo_sk", "c_current_addr_sk", "c_first_shipto_date_sk", "c_first_sales_date_sk", "c_salutation", "c_first_name", "c_last_name", "c_preferred_cust_flag", "c_birth_day", "c_birth_month", "c_birth_year", "c_birth_country", "c_login", "c_email_address", "c_last_review_date"
+FROM "CUSTOMER"
+WHERE "c_current_addr_sk" IS NOT NULL AND "c_current_cdemo_sk" IS NOT NULL AND "c_customer_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "ca_address_sk", "ca_address_id", "ca_street_number", "ca_street_name", "ca_street_type", "ca_suite_number", "ca_city", "ca_county", "ca_state", "ca_zip", "ca_country", "ca_gmt_offset", "ca_location_type"
+FROM "CUSTOMER_ADDRESS"
+WHERE "ca_state" IN ('CO', 'IL', 'MN') AND "ca_address_sk" IS NOT NULL) AS "t2" ON "t0"."c_current_addr_sk" = "t2"."ca_address_sk"
+INNER JOIN (SELECT "cd_demo_sk", "cd_gender", "cd_marital_status", "cd_education_status", "cd_purchase_estimate", "cd_credit_rating", "cd_dep_count", "cd_dep_employed_count", "cd_dep_college_count"
+FROM "CUSTOMER_DEMOGRAPHICS"
+WHERE "cd_demo_sk" IS NOT NULL) AS "t4" ON "t0"."c_current_cdemo_sk" = "t4"."cd_demo_sk"
+                    hive.sql.query.fieldNames c_customer_sk,c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ca_address_sk,ca_address_id,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_locati [...]
+                    hive.sql.query.fieldTypes int,char(16),int,int,int,int,int,char(10),char(20),char(30),char(1),int,int,int,varchar(20),char(13),char(50),char(10),int,char(16),char(10),varchar(60),char(15),char(10),varchar(60),varchar(30),char(2),char(10),varchar(20),decimal(5,2),char(20),int,char(1),char(1),char(20),int,char(10),int,int,int
+                    hive.sql.query.split false
+                  Statistics: Num rows: 1 Data size: 376 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: c_customer_sk (type: int), cd_gender (type: char(1)), cd_marital_status (type: char(1)), cd_education_status (type: char(20)), cd_purchase_estimate (type: int), cd_credit_rating (type: char(10))
+                    outputColumnNames: _col0, _col32, _col33, _col34, _col35, _col36
+                    Statistics: Num rows: 1 Data size: 376 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: int)
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: int)
+                      Statistics: Num rows: 1 Data size: 376 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Map 7 
+            Map Operator Tree:
+                TableScan
+                  alias: store_sales
+                  properties:
+                    hive.sql.query SELECT "t0"."ss_customer_sk" AS "ss_customer_sk0"
+FROM (SELECT "ss_sold_date_sk", "ss_sold_time_sk", "ss_item_sk", "ss_customer_sk", "ss_cdemo_sk", "ss_hdemo_sk", "ss_addr_sk", "ss_store_sk", "ss_promo_sk", "ss_ticket_number", "ss_quantity", "ss_wholesale_cost", "ss_list_price", "ss_sales_price", "ss_ext_discount_amt", "ss_ext_sales_price", "ss_ext_wholesale_cost", "ss_ext_list_price", "ss_ext_tax", "ss_coupon_amt", "ss_net_paid", "ss_net_paid_inc_tax", "ss_net_profit"
+FROM "STORE_SALES"
+WHERE "ss_customer_sk" IS NOT NULL AND "ss_sold_date_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "d_date_sk"
+FROM "DATE_DIM"
+WHERE "d_year" = 1999 AND "d_moy" BETWEEN 1 AND 3 AND "d_date_sk" IS NOT NULL) AS "t2" ON "t0"."ss_sold_date_sk" = "t2"."d_date_sk"
+                    hive.sql.query.fieldNames ss_customer_sk0
+                    hive.sql.query.fieldTypes int
+                    hive.sql.query.split false
+                  Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: ss_customer_sk0 (type: int)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                    Group By Operator
+                      keys: _col0 (type: int)
+                      minReductionHashAggr: 0.99
+                      mode: hash
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Map 8 
+            Map Operator Tree:
+                TableScan
+                  alias: web_sales
+                  properties:
+                    hive.sql.query SELECT "t0"."ws_bill_customer_sk" AS "ws_bill_customer_sk0", TRUE AS "$f1"
+FROM (SELECT "ws_sold_date_sk", "ws_sold_time_sk", "ws_ship_date_sk", "ws_item_sk", "ws_bill_customer_sk", "ws_bill_cdemo_sk", "ws_bill_hdemo_sk", "ws_bill_addr_sk", "ws_ship_customer_sk", "ws_ship_cdemo_sk", "ws_ship_hdemo_sk", "ws_ship_addr_sk", "ws_web_page_sk", "ws_web_site_sk", "ws_ship_mode_sk", "ws_warehouse_sk", "ws_promo_sk", "ws_order_number", "ws_quantity", "ws_wholesale_cost", "ws_list_price", "ws_sales_price", "ws_ext_discount_amt", "ws_ext_sales_price", "ws_ext_wholesale_co [...]
+FROM "WEB_SALES"
+WHERE "ws_bill_customer_sk" IS NOT NULL AND "ws_sold_date_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "d_date_sk"
+FROM "DATE_DIM"
+WHERE "d_year" = 1999 AND "d_moy" BETWEEN 1 AND 3 AND "d_date_sk" IS NOT NULL) AS "t2" ON "t0"."ws_sold_date_sk" = "t2"."d_date_sk"
+GROUP BY "t0"."ws_bill_customer_sk"
+                    hive.sql.query.fieldNames ws_bill_customer_sk0,$f1
+                    hive.sql.query.fieldTypes int,boolean
+                    hive.sql.query.split false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: ws_bill_customer_sk0 (type: int), $f1 (type: boolean)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: int)
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: int)
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col1 (type: boolean)
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Map 9 
+            Map Operator Tree:
+                TableScan
+                  alias: catalog_sales
+                  properties:
+                    hive.sql.query SELECT "t0"."cs_ship_customer_sk" AS "cs_ship_customer_sk0", TRUE AS "$f1"
+FROM (SELECT "cs_sold_date_sk", "cs_sold_time_sk", "cs_ship_date_sk", "cs_bill_customer_sk", "cs_bill_cdemo_sk", "cs_bill_hdemo_sk", "cs_bill_addr_sk", "cs_ship_customer_sk", "cs_ship_cdemo_sk", "cs_ship_hdemo_sk", "cs_ship_addr_sk", "cs_call_center_sk", "cs_catalog_page_sk", "cs_ship_mode_sk", "cs_warehouse_sk", "cs_item_sk", "cs_promo_sk", "cs_order_number", "cs_quantity", "cs_wholesale_cost", "cs_list_price", "cs_sales_price", "cs_ext_discount_amt", "cs_ext_sales_price", "cs_ext_whole [...]
+FROM "CATALOG_SALES"
+WHERE "cs_ship_customer_sk" IS NOT NULL AND "cs_sold_date_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "d_date_sk"
+FROM "DATE_DIM"
+WHERE "d_year" = 1999 AND "d_moy" BETWEEN 1 AND 3 AND "d_date_sk" IS NOT NULL) AS "t2" ON "t0"."cs_sold_date_sk" = "t2"."d_date_sk"
+GROUP BY "t0"."cs_ship_customer_sk"
+                    hive.sql.query.fieldNames cs_ship_customer_sk0,$f1
+                    hive.sql.query.fieldTypes int,boolean
+                    hive.sql.query.split false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: cs_ship_customer_sk0 (type: int), $f1 (type: boolean)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: int)
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: int)
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col1 (type: boolean)
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Left Semi Join 0 to 1
+                keys:
+                  0 _col0 (type: int)
+                  1 _col0 (type: int)
+                outputColumnNames: _col0, _col32, _col33, _col34, _col35, _col36
+                Statistics: Num rows: 1 Data size: 413 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col0 (type: int)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: int)
+                  Statistics: Num rows: 1 Data size: 413 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+        Reducer 3 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Left Outer Join 0 to 1
+                keys:
+                  0 _col0 (type: int)
+                  1 _col0 (type: int)
+                outputColumnNames: _col0, _col32, _col33, _col34, _col35, _col36, _col41
+                Statistics: Num rows: 1 Data size: 454 Basic stats: COMPLETE Column stats: NONE
+                Filter Operator
+                  predicate: _col41 is null (type: boolean)
+                  Statistics: Num rows: 1 Data size: 454 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: _col0 (type: int), _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+                    outputColumnNames: _col0, _col32, _col33, _col34, _col35, _col36
+                    Statistics: Num rows: 1 Data size: 454 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: int)
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: int)
+                      Statistics: Num rows: 1 Data size: 454 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+        Reducer 4 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Left Outer Join 0 to 1
+                keys:
+                  0 _col0 (type: int)
+                  1 _col0 (type: int)
+                outputColumnNames: _col32, _col33, _col34, _col35, _col36, _col43
+                Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                Filter Operator
+                  predicate: _col43 is null (type: boolean)
+                  Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+                    outputColumnNames: _col32, _col33, _col34, _col35, _col36
+                    Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                    Top N Key Operator
+                      sort order: +++++
+                      keys: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+                      Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                      top n: 100
+                      Group By Operator
+                        aggregations: count()
+                        keys: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+                        minReductionHashAggr: 0.99
+                        mode: hash
+                        outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+                        Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                        Reduce Output Operator
+                          key expressions: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col3 (type: int), _col4 (type: char(10))
+                          sort order: +++++
+                          Map-reduce partition columns: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col3 (type: int), _col4 (type: char(10))
+                          Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                          TopN Hash Memory Usage: 0.1
+                          value expressions: _col5 (type: bigint)
+        Reducer 5 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                keys: KEY._col0 (type: char(1)), KEY._col1 (type: char(1)), KEY._col2 (type: char(20)), KEY._col3 (type: int), KEY._col4 (type: char(10))
+                mode: mergepartial
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+                Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                Select Operator
+                  expressions: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col5 (type: bigint), _col3 (type: int), _col4 (type: char(10))
+                  outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col6
+                  Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col4 (type: int), _col6 (type: char(10))
+                    sort order: +++++
+                    Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                    TopN Hash Memory Usage: 0.1
+                    value expressions: _col3 (type: bigint)
+        Reducer 6 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: KEY.reducesinkkey0 (type: char(1)), KEY.reducesinkkey1 (type: char(1)), KEY.reducesinkkey2 (type: char(20)), VALUE._col0 (type: bigint), KEY.reducesinkkey3 (type: int), VALUE._col0 (type: bigint), KEY.reducesinkkey4 (type: char(10)), VALUE._col0 (type: bigint)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
+                Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                Limit
+                  Number of rows: 100
+                  Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                  File Output Operator
+                    compressed: false
+                    Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                    table:
+                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: 100
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT cd_gender, 
+       cd_marital_status, 
+       cd_education_status, 
+       Count(*) cnt1, 
+       cd_purchase_estimate, 
+       Count(*) cnt2, 
+       cd_credit_rating, 
+       Count(*) cnt3 
+FROM   customer c, 
+       customer_address ca, 
+       customer_demographics 
+WHERE  c.c_current_addr_sk = ca.ca_address_sk 
+       AND ca_state IN ( 'CO', 'IL', 'MN' ) 
+       AND cd_demo_sk = c.c_current_cdemo_sk 
+       AND EXISTS (SELECT * 
+                   FROM   store_sales, 
+                          date_dim 
+                   WHERE  c.c_customer_sk = ss_customer_sk 
+                          AND ss_sold_date_sk = d_date_sk 
+                          AND d_year = 1999 
+                          AND d_moy BETWEEN 1 AND 1 + 2) 
+       AND ( NOT EXISTS (SELECT * 
+                         FROM   web_sales, 
+                                date_dim 
+                         WHERE  c.c_customer_sk = ws_bill_customer_sk 
+                                AND ws_sold_date_sk = d_date_sk 
+                                AND d_year = 1999 
+                                AND d_moy BETWEEN 1 AND 1 + 2) 
+             AND NOT EXISTS (SELECT * 
+                             FROM   catalog_sales, 
+                                    date_dim 
+                             WHERE  c.c_customer_sk = cs_ship_customer_sk 
+                                    AND cs_sold_date_sk = d_date_sk 
+                                    AND d_year = 1999 
+                                    AND d_moy BETWEEN 1 AND 1 + 2) ) 
+GROUP  BY cd_gender, 
+          cd_marital_status, 
+          cd_education_status, 
+          cd_purchase_estimate, 
+          cd_credit_rating 
+ORDER  BY cd_gender, 
+          cd_marital_status, 
+          cd_education_status, 
+          cd_purchase_estimate, 
+          cd_credit_rating 
+LIMIT  100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@catalog_sales
+PREHOOK: Input: default@customer
+PREHOOK: Input: default@customer_address
+PREHOOK: Input: default@customer_demographics
+PREHOOK: Input: default@date_dim
+PREHOOK: Input: default@store_sales
+PREHOOK: Input: default@web_sales
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT cd_gender, 
+       cd_marital_status, 
+       cd_education_status, 
+       Count(*) cnt1, 
+       cd_purchase_estimate, 
+       Count(*) cnt2, 
+       cd_credit_rating, 
+       Count(*) cnt3 
+FROM   customer c, 
+       customer_address ca, 
+       customer_demographics 
+WHERE  c.c_current_addr_sk = ca.ca_address_sk 
+       AND ca_state IN ( 'CO', 'IL', 'MN' ) 
+       AND cd_demo_sk = c.c_current_cdemo_sk 
+       AND EXISTS (SELECT * 
+                   FROM   store_sales, 
+                          date_dim 
+                   WHERE  c.c_customer_sk = ss_customer_sk 
+                          AND ss_sold_date_sk = d_date_sk 
+                          AND d_year = 1999 
+                          AND d_moy BETWEEN 1 AND 1 + 2) 
+       AND ( NOT EXISTS (SELECT * 
+                         FROM   web_sales, 
+                                date_dim 
+                         WHERE  c.c_customer_sk = ws_bill_customer_sk 
+                                AND ws_sold_date_sk = d_date_sk 
+                                AND d_year = 1999 
+                                AND d_moy BETWEEN 1 AND 1 + 2) 
+             AND NOT EXISTS (SELECT * 
+                             FROM   catalog_sales, 
+                                    date_dim 
+                             WHERE  c.c_customer_sk = cs_ship_customer_sk 
+                                    AND cs_sold_date_sk = d_date_sk 
+                                    AND d_year = 1999 
+                                    AND d_moy BETWEEN 1 AND 1 + 2) ) 
+GROUP  BY cd_gender, 
+          cd_marital_status, 
+          cd_education_status, 
+          cd_purchase_estimate, 
+          cd_credit_rating 
+ORDER  BY cd_gender, 
+          cd_marital_status, 
+          cd_education_status, 
+          cd_purchase_estimate, 
+          cd_credit_rating 
+LIMIT  100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@catalog_sales
+POSTHOOK: Input: default@customer
+POSTHOOK: Input: default@customer_address
+POSTHOOK: Input: default@customer_demographics
+POSTHOOK: Input: default@date_dim
+POSTHOOK: Input: default@store_sales
+POSTHOOK: Input: default@web_sales
+#### A masked pattern was here ####
+PREHOOK: query: explain
+SELECT cd_gender, 
+       cd_marital_status, 
+       cd_education_status, 
+       Count(*) cnt1, 
+       cd_purchase_estimate, 
+       Count(*) cnt2, 
+       cd_credit_rating, 
+       Count(*) cnt3 
+FROM   customer c, 
+       customer_address ca, 
+       customer_demographics 
+WHERE  c.c_current_addr_sk = ca.ca_address_sk 
+       AND ca_state IN ( 'CO', 'IL', 'MN' ) 
+       AND cd_demo_sk = c.c_current_cdemo_sk 
+       AND EXISTS (SELECT * 
+                   FROM   store_sales, 
+                          date_dim 
+                   WHERE  c.c_customer_sk = ss_customer_sk 
+                          AND ss_sold_date_sk = d_date_sk 
+                          AND d_year = 1999 
+                          AND d_moy NOT BETWEEN 1 AND 1 + 2) 
+       AND ( NOT EXISTS (SELECT * 
+                         FROM   web_sales, 
+                                date_dim 
+                         WHERE  c.c_customer_sk = ws_bill_customer_sk 
+                                AND ws_sold_date_sk = d_date_sk 
+                                AND d_year = 1999 
+                                AND d_moy NOT BETWEEN 1 AND 1 + 2) 
+             AND NOT EXISTS (SELECT * 
+                             FROM   catalog_sales, 
+                                    date_dim 
+                             WHERE  c.c_customer_sk = cs_ship_customer_sk 
+                                    AND cs_sold_date_sk = d_date_sk 
+                                    AND d_year = 1999 
+                                    AND d_moy NOT BETWEEN 1 AND 1 + 2) ) 
+GROUP  BY cd_gender, 
+          cd_marital_status, 
+          cd_education_status, 
+          cd_purchase_estimate, 
+          cd_credit_rating 
+ORDER  BY cd_gender, 
+          cd_marital_status, 
+          cd_education_status, 
+          cd_purchase_estimate, 
+          cd_credit_rating 
+LIMIT  100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@catalog_sales
+PREHOOK: Input: default@customer
+PREHOOK: Input: default@customer_address
+PREHOOK: Input: default@customer_demographics
+PREHOOK: Input: default@date_dim
+PREHOOK: Input: default@store_sales
+PREHOOK: Input: default@web_sales
+#### A masked pattern was here ####
+POSTHOOK: query: explain
+SELECT cd_gender, 
+       cd_marital_status, 
+       cd_education_status, 
+       Count(*) cnt1, 
+       cd_purchase_estimate, 
+       Count(*) cnt2, 
+       cd_credit_rating, 
+       Count(*) cnt3 
+FROM   customer c, 
+       customer_address ca, 
+       customer_demographics 
+WHERE  c.c_current_addr_sk = ca.ca_address_sk 
+       AND ca_state IN ( 'CO', 'IL', 'MN' ) 
+       AND cd_demo_sk = c.c_current_cdemo_sk 
+       AND EXISTS (SELECT * 
+                   FROM   store_sales, 
+                          date_dim 
+                   WHERE  c.c_customer_sk = ss_customer_sk 
+                          AND ss_sold_date_sk = d_date_sk 
+                          AND d_year = 1999 
+                          AND d_moy NOT BETWEEN 1 AND 1 + 2) 
+       AND ( NOT EXISTS (SELECT * 
+                         FROM   web_sales, 
+                                date_dim 
+                         WHERE  c.c_customer_sk = ws_bill_customer_sk 
+                                AND ws_sold_date_sk = d_date_sk 
+                                AND d_year = 1999 
+                                AND d_moy NOT BETWEEN 1 AND 1 + 2) 
+             AND NOT EXISTS (SELECT * 
+                             FROM   catalog_sales, 
+                                    date_dim 
+                             WHERE  c.c_customer_sk = cs_ship_customer_sk 
+                                    AND cs_sold_date_sk = d_date_sk 
+                                    AND d_year = 1999 
+                                    AND d_moy NOT BETWEEN 1 AND 1 + 2) ) 
+GROUP  BY cd_gender, 
+          cd_marital_status, 
+          cd_education_status, 
+          cd_purchase_estimate, 
+          cd_credit_rating 
+ORDER  BY cd_gender, 
+          cd_marital_status, 
+          cd_education_status, 
+          cd_purchase_estimate, 
+          cd_credit_rating 
+LIMIT  100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@catalog_sales
+POSTHOOK: Input: default@customer
+POSTHOOK: Input: default@customer_address
+POSTHOOK: Input: default@customer_demographics
+POSTHOOK: Input: default@date_dim
+POSTHOOK: Input: default@store_sales
+POSTHOOK: Input: default@web_sales
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
+        Reducer 3 <- Map 8 (SIMPLE_EDGE), Reducer 2 (ONE_TO_ONE_EDGE)
+        Reducer 4 <- Map 9 (SIMPLE_EDGE), Reducer 3 (ONE_TO_ONE_EDGE)
+        Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
+        Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: c
+                  properties:
+                    hive.sql.query SELECT "t0"."c_customer_sk", "t0"."c_customer_id", "t0"."c_current_cdemo_sk", "t0"."c_current_hdemo_sk", "t0"."c_current_addr_sk", "t0"."c_first_shipto_date_sk", "t0"."c_first_sales_date_sk", "t0"."c_salutation", "t0"."c_first_name", "t0"."c_last_name", "t0"."c_preferred_cust_flag", "t0"."c_birth_day", "t0"."c_birth_month", "t0"."c_birth_year", "t0"."c_birth_country", "t0"."c_login", "t0"."c_email_address", "t0"."c_last_review_date", "t2"."ca_address_sk [...]
+FROM (SELECT "c_customer_sk", "c_customer_id", "c_current_cdemo_sk", "c_current_hdemo_sk", "c_current_addr_sk", "c_first_shipto_date_sk", "c_first_sales_date_sk", "c_salutation", "c_first_name", "c_last_name", "c_preferred_cust_flag", "c_birth_day", "c_birth_month", "c_birth_year", "c_birth_country", "c_login", "c_email_address", "c_last_review_date"
+FROM "CUSTOMER"
+WHERE "c_current_addr_sk" IS NOT NULL AND "c_current_cdemo_sk" IS NOT NULL AND "c_customer_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "ca_address_sk", "ca_address_id", "ca_street_number", "ca_street_name", "ca_street_type", "ca_suite_number", "ca_city", "ca_county", "ca_state", "ca_zip", "ca_country", "ca_gmt_offset", "ca_location_type"
+FROM "CUSTOMER_ADDRESS"
+WHERE "ca_state" IN ('CO', 'IL', 'MN') AND "ca_address_sk" IS NOT NULL) AS "t2" ON "t0"."c_current_addr_sk" = "t2"."ca_address_sk"
+INNER JOIN (SELECT "cd_demo_sk", "cd_gender", "cd_marital_status", "cd_education_status", "cd_purchase_estimate", "cd_credit_rating", "cd_dep_count", "cd_dep_employed_count", "cd_dep_college_count"
+FROM "CUSTOMER_DEMOGRAPHICS"
+WHERE "cd_demo_sk" IS NOT NULL) AS "t4" ON "t0"."c_current_cdemo_sk" = "t4"."cd_demo_sk"
+                    hive.sql.query.fieldNames c_customer_sk,c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ca_address_sk,ca_address_id,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_locati [...]
+                    hive.sql.query.fieldTypes int,char(16),int,int,int,int,int,char(10),char(20),char(30),char(1),int,int,int,varchar(20),char(13),char(50),char(10),int,char(16),char(10),varchar(60),char(15),char(10),varchar(60),varchar(30),char(2),char(10),varchar(20),decimal(5,2),char(20),int,char(1),char(1),char(20),int,char(10),int,int,int
+                    hive.sql.query.split false
+                  Statistics: Num rows: 1 Data size: 376 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: c_customer_sk (type: int), cd_gender (type: char(1)), cd_marital_status (type: char(1)), cd_education_status (type: char(20)), cd_purchase_estimate (type: int), cd_credit_rating (type: char(10))
+                    outputColumnNames: _col0, _col32, _col33, _col34, _col35, _col36
+                    Statistics: Num rows: 1 Data size: 376 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: int)
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: int)
+                      Statistics: Num rows: 1 Data size: 376 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Map 7 
+            Map Operator Tree:
+                TableScan
+                  alias: store_sales
+                  properties:
+                    hive.sql.query SELECT "t0"."ss_customer_sk" AS "ss_customer_sk0"
+FROM (SELECT "ss_sold_date_sk", "ss_sold_time_sk", "ss_item_sk", "ss_customer_sk", "ss_cdemo_sk", "ss_hdemo_sk", "ss_addr_sk", "ss_store_sk", "ss_promo_sk", "ss_ticket_number", "ss_quantity", "ss_wholesale_cost", "ss_list_price", "ss_sales_price", "ss_ext_discount_amt", "ss_ext_sales_price", "ss_ext_wholesale_cost", "ss_ext_list_price", "ss_ext_tax", "ss_coupon_amt", "ss_net_paid", "ss_net_paid_inc_tax", "ss_net_profit"
+FROM "STORE_SALES"
+WHERE "ss_customer_sk" IS NOT NULL AND "ss_sold_date_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "d_date_sk"
+FROM "DATE_DIM"
+WHERE "d_moy" BETWEEN 1 AND 3 AND "d_year" = 1999 AND "d_date_sk" IS NOT NULL) AS "t2" ON "t0"."ss_sold_date_sk" = "t2"."d_date_sk"
+                    hive.sql.query.fieldNames ss_customer_sk0
+                    hive.sql.query.fieldTypes int
+                    hive.sql.query.split false
+                  Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: ss_customer_sk0 (type: int)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                    Group By Operator
+                      keys: _col0 (type: int)
+                      minReductionHashAggr: 0.99
+                      mode: hash
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Map 8 
+            Map Operator Tree:
+                TableScan
+                  alias: web_sales
+                  properties:
+                    hive.sql.query SELECT "t0"."ws_bill_customer_sk" AS "ws_bill_customer_sk0", TRUE AS "$f1"
+FROM (SELECT "ws_sold_date_sk", "ws_sold_time_sk", "ws_ship_date_sk", "ws_item_sk", "ws_bill_customer_sk", "ws_bill_cdemo_sk", "ws_bill_hdemo_sk", "ws_bill_addr_sk", "ws_ship_customer_sk", "ws_ship_cdemo_sk", "ws_ship_hdemo_sk", "ws_ship_addr_sk", "ws_web_page_sk", "ws_web_site_sk", "ws_ship_mode_sk", "ws_warehouse_sk", "ws_promo_sk", "ws_order_number", "ws_quantity", "ws_wholesale_cost", "ws_list_price", "ws_sales_price", "ws_ext_discount_amt", "ws_ext_sales_price", "ws_ext_wholesale_co [...]
+FROM "WEB_SALES"
+WHERE "ws_bill_customer_sk" IS NOT NULL AND "ws_sold_date_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "d_date_sk"
+FROM "DATE_DIM"
+WHERE "d_moy" BETWEEN 1 AND 3 AND "d_year" = 1999 AND "d_date_sk" IS NOT NULL) AS "t2" ON "t0"."ws_sold_date_sk" = "t2"."d_date_sk"
+GROUP BY "t0"."ws_bill_customer_sk"
+                    hive.sql.query.fieldNames ws_bill_customer_sk0,$f1
+                    hive.sql.query.fieldTypes int,boolean
+                    hive.sql.query.split false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: ws_bill_customer_sk0 (type: int), $f1 (type: boolean)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: int)
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: int)
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col1 (type: boolean)
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Map 9 
+            Map Operator Tree:
+                TableScan
+                  alias: catalog_sales
+                  properties:
+                    hive.sql.query SELECT "t0"."cs_ship_customer_sk" AS "cs_ship_customer_sk0", TRUE AS "$f1"
+FROM (SELECT "cs_sold_date_sk", "cs_sold_time_sk", "cs_ship_date_sk", "cs_bill_customer_sk", "cs_bill_cdemo_sk", "cs_bill_hdemo_sk", "cs_bill_addr_sk", "cs_ship_customer_sk", "cs_ship_cdemo_sk", "cs_ship_hdemo_sk", "cs_ship_addr_sk", "cs_call_center_sk", "cs_catalog_page_sk", "cs_ship_mode_sk", "cs_warehouse_sk", "cs_item_sk", "cs_promo_sk", "cs_order_number", "cs_quantity", "cs_wholesale_cost", "cs_list_price", "cs_sales_price", "cs_ext_discount_amt", "cs_ext_sales_price", "cs_ext_whole [...]
+FROM "CATALOG_SALES"
+WHERE "cs_ship_customer_sk" IS NOT NULL AND "cs_sold_date_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "d_date_sk"
+FROM "DATE_DIM"
+WHERE "d_moy" BETWEEN 1 AND 3 AND "d_year" = 1999 AND "d_date_sk" IS NOT NULL) AS "t2" ON "t0"."cs_sold_date_sk" = "t2"."d_date_sk"
+GROUP BY "t0"."cs_ship_customer_sk"
+                    hive.sql.query.fieldNames cs_ship_customer_sk0,$f1
+                    hive.sql.query.fieldTypes int,boolean
+                    hive.sql.query.split false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: cs_ship_customer_sk0 (type: int), $f1 (type: boolean)
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: int)
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: int)
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col1 (type: boolean)
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Left Semi Join 0 to 1
+                keys:
+                  0 _col0 (type: int)
+                  1 _col0 (type: int)
+                outputColumnNames: _col0, _col32, _col33, _col34, _col35, _col36
+                Statistics: Num rows: 1 Data size: 413 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col0 (type: int)
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: int)
+                  Statistics: Num rows: 1 Data size: 413 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+        Reducer 3 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Left Outer Join 0 to 1
+                keys:
+                  0 _col0 (type: int)
+                  1 _col0 (type: int)
+                outputColumnNames: _col0, _col32, _col33, _col34, _col35, _col36, _col41
+                Statistics: Num rows: 1 Data size: 454 Basic stats: COMPLETE Column stats: NONE
+                Filter Operator
+                  predicate: _col41 is null (type: boolean)
+                  Statistics: Num rows: 1 Data size: 454 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: _col0 (type: int), _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+                    outputColumnNames: _col0, _col32, _col33, _col34, _col35, _col36
+                    Statistics: Num rows: 1 Data size: 454 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: int)
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: int)
+                      Statistics: Num rows: 1 Data size: 454 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+        Reducer 4 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Left Outer Join 0 to 1
+                keys:
+                  0 _col0 (type: int)
+                  1 _col0 (type: int)
+                outputColumnNames: _col32, _col33, _col34, _col35, _col36, _col43
+                Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                Filter Operator
+                  predicate: _col43 is null (type: boolean)
+                  Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+                    outputColumnNames: _col32, _col33, _col34, _col35, _col36
+                    Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                    Top N Key Operator
+                      sort order: +++++
+                      keys: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+                      Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                      top n: 100
+                      Group By Operator
+                        aggregations: count()
+                        keys: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+                        minReductionHashAggr: 0.99
+                        mode: hash
+                        outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+                        Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                        Reduce Output Operator
+                          key expressions: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col3 (type: int), _col4 (type: char(10))
+                          sort order: +++++
+                          Map-reduce partition columns: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col3 (type: int), _col4 (type: char(10))
+                          Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                          TopN Hash Memory Usage: 0.1
+                          value expressions: _col5 (type: bigint)
+        Reducer 5 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                keys: KEY._col0 (type: char(1)), KEY._col1 (type: char(1)), KEY._col2 (type: char(20)), KEY._col3 (type: int), KEY._col4 (type: char(10))
+                mode: mergepartial
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+                Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                Select Operator
+                  expressions: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col5 (type: bigint), _col3 (type: int), _col4 (type: char(10))
+                  outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col6
+                  Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col4 (type: int), _col6 (type: char(10))
+                    sort order: +++++
+                    Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                    TopN Hash Memory Usage: 0.1
+                    value expressions: _col3 (type: bigint)
+        Reducer 6 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: KEY.reducesinkkey0 (type: char(1)), KEY.reducesinkkey1 (type: char(1)), KEY.reducesinkkey2 (type: char(20)), VALUE._col0 (type: bigint), KEY.reducesinkkey3 (type: int), VALUE._col0 (type: bigint), KEY.reducesinkkey4 (type: char(10)), VALUE._col0 (type: bigint)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
+                Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                Limit
+                  Number of rows: 100
+                  Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                  File Output Operator
+                    compressed: false
+                    Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+                    table:
+                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: 100
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT cd_gender, 
+       cd_marital_status, 
+       cd_education_status, 
+       Count(*) cnt1, 
+       cd_purchase_estimate, 
+       Count(*) cnt2, 
+       cd_credit_rating, 
+       Count(*) cnt3 
+FROM   customer c, 
+       customer_address ca, 
+       customer_demographics 
+WHERE  c.c_current_addr_sk = ca.ca_address_sk 
+       AND ca_state IN ( 'CO', 'IL', 'MN' ) 
+       AND cd_demo_sk = c.c_current_cdemo_sk 
+       AND EXISTS (SELECT * 
+                   FROM   store_sales, 
+                          date_dim 
+                   WHERE  c.c_customer_sk = ss_customer_sk 
+                          AND ss_sold_date_sk = d_date_sk 
+                          AND d_year = 1999 
+                          AND d_moy NOT BETWEEN 1 AND 1 + 2) 
+       AND ( NOT EXISTS (SELECT * 
+                         FROM   web_sales, 
+                                date_dim 
+                         WHERE  c.c_customer_sk = ws_bill_customer_sk 
+                                AND ws_sold_date_sk = d_date_sk 
+                                AND d_year = 1999 
+                                AND d_moy NOT BETWEEN 1 AND 1 + 2) 
+             AND NOT EXISTS (SELECT * 
+                             FROM   catalog_sales, 
+                                    date_dim 
+                             WHERE  c.c_customer_sk = cs_ship_customer_sk 
+                                    AND cs_sold_date_sk = d_date_sk 
+                                    AND d_year = 1999 
+                                    AND d_moy NOT BETWEEN 1 AND 1 + 2) ) 
+GROUP  BY cd_gender, 
+          cd_marital_status, 
+          cd_education_status, 
+          cd_purchase_estimate, 
+          cd_credit_rating 
+ORDER  BY cd_gender, 
+          cd_marital_status, 
+          cd_education_status, 
+          cd_purchase_estimate, 
+          cd_credit_rating 
+LIMIT  100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@catalog_sales
+PREHOOK: Input: default@customer
+PREHOOK: Input: default@customer_address
+PREHOOK: Input: default@customer_demographics
+PREHOOK: Input: default@date_dim
+PREHOOK: Input: default@store_sales
+PREHOOK: Input: default@web_sales
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT cd_gender, 
+       cd_marital_status, 
+       cd_education_status, 
+       Count(*) cnt1, 
+       cd_purchase_estimate, 
+       Count(*) cnt2, 
+       cd_credit_rating, 
+       Count(*) cnt3 
+FROM   customer c, 
+       customer_address ca, 
+       customer_demographics 
+WHERE  c.c_current_addr_sk = ca.ca_address_sk 
+       AND ca_state IN ( 'CO', 'IL', 'MN' ) 
+       AND cd_demo_sk = c.c_current_cdemo_sk 
+       AND EXISTS (SELECT * 
+                   FROM   store_sales, 
+                          date_dim 
+                   WHERE  c.c_customer_sk = ss_customer_sk 
+                          AND ss_sold_date_sk = d_date_sk 
+                          AND d_year = 1999 
+                          AND d_moy NOT BETWEEN 1 AND 1 + 2) 
+       AND ( NOT EXISTS (SELECT * 
+                         FROM   web_sales, 
+                                date_dim 
+                         WHERE  c.c_customer_sk = ws_bill_customer_sk 
+                                AND ws_sold_date_sk = d_date_sk 
+                                AND d_year = 1999 
+                                AND d_moy NOT BETWEEN 1 AND 1 + 2) 
+             AND NOT EXISTS (SELECT * 
+                             FROM   catalog_sales, 
+                                    date_dim 
+                             WHERE  c.c_customer_sk = cs_ship_customer_sk 
+                                    AND cs_sold_date_sk = d_date_sk 
+                                    AND d_year = 1999 
+                                    AND d_moy NOT BETWEEN 1 AND 1 + 2) ) 
+GROUP  BY cd_gender, 
+          cd_marital_status, 
+          cd_education_status, 
+          cd_purchase_estimate, 
+          cd_credit_rating 
+ORDER  BY cd_gender, 
+          cd_marital_status, 
+          cd_education_status, 
+          cd_purchase_estimate, 
+          cd_credit_rating 
+LIMIT  100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@catalog_sales
+POSTHOOK: Input: default@customer
+POSTHOOK: Input: default@customer_address
+POSTHOOK: Input: default@customer_demographics
+POSTHOOK: Input: default@date_dim
+POSTHOOK: Input: default@store_sales
+POSTHOOK: Input: default@web_sales
+#### A masked pattern was here ####
+PREHOOK: query: explain
+SELECT Count(*) 
+FROM   (SELECT DISTINCT c_last_name, 
+                        c_first_name, 
+                        d_date 
+        FROM   store_sales, 
+               date_dim, 
+               customer 
+        WHERE  store_sales.ss_sold_date_sk = date_dim.d_date_sk 
+               AND store_sales.ss_customer_sk = customer.c_customer_sk 
+               AND d_month_seq BETWEEN 1212 AND 1212 + 11 
+        intersect 
+        SELECT DISTINCT c_last_name, 
+                        c_first_name, 
+                        d_date 
+        FROM   catalog_sales, 
+               date_dim, 
+               customer 
+        WHERE  catalog_sales.cs_sold_date_sk = date_dim.d_date_sk 
+               AND catalog_sales.cs_bill_customer_sk = customer.c_customer_sk 
+               AND d_month_seq BETWEEN 1212 AND 1212 + 11 
+        intersect 
+        SELECT DISTINCT c_last_name, 
+                        c_first_name, 
+                        d_date 
+        FROM   web_sales, 
+               date_dim, 
+               customer 
+        WHERE  web_sales.ws_sold_date_sk = date_dim.d_date_sk 
+               AND web_sales.ws_bill_customer_sk = customer.c_customer_sk 
+               AND d_month_seq BETWEEN 1212 AND 1212 + 11) hot_cust 
+LIMIT  100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@catalog_sales
+PREHOOK: Input: default@customer
+PREHOOK: Input: default@date_dim
+PREHOOK: Input: default@store_sales
+PREHOOK: Input: default@web_sales
+#### A masked pattern was here ####
+POSTHOOK: query: explain
+SELECT Count(*) 
+FROM   (SELECT DISTINCT c_last_name, 
+                        c_first_name, 
+                        d_date 
+        FROM   store_sales, 
+               date_dim, 
+               customer 
+        WHERE  store_sales.ss_sold_date_sk = date_dim.d_date_sk 
+               AND store_sales.ss_customer_sk = customer.c_customer_sk 
+               AND d_month_seq BETWEEN 1212 AND 1212 + 11 
+        intersect 
+        SELECT DISTINCT c_last_name, 
+                        c_first_name, 
+                        d_date 
+        FROM   catalog_sales, 
+               date_dim, 
+               customer 
+        WHERE  catalog_sales.cs_sold_date_sk = date_dim.d_date_sk 
+               AND catalog_sales.cs_bill_customer_sk = customer.c_customer_sk 
+               AND d_month_seq BETWEEN 1212 AND 1212 + 11 
+        intersect 
+        SELECT DISTINCT c_last_name, 
+                        c_first_name, 
+                        d_date 
+        FROM   web_sales, 
+               date_dim, 
+               customer 
+        WHERE  web_sales.ws_sold_date_sk = date_dim.d_date_sk 
+               AND web_sales.ws_bill_customer_sk = customer.c_customer_sk 
+               AND d_month_seq BETWEEN 1212 AND 1212 + 11) hot_cust 
+LIMIT  100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@catalog_sales
+POSTHOOK: Input: default@customer
+POSTHOOK: Input: default@date_dim
+POSTHOOK: Input: default@store_sales
+POSTHOOK: Input: default@web_sales
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Map 1 <- Union 2 (CONTAINS)
+        Map 5 <- Union 2 (CONTAINS)
+        Map 6 <- Union 2 (CONTAINS)
+        Reducer 3 <- Union 2 (SIMPLE_EDGE)
+        Reducer 4 <- Reducer 3 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: store_sales
+                  Statistics: Num rows: 1 Data size: 282 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: c_last_name (type: char(30)), c_first_name (type: char(20)), d_date (type: date), $f3 (type: bigint)
+                    outputColumnNames: _col0, _col1, _col2, _col3
+                    Statistics: Num rows: 1 Data size: 282 Basic stats: COMPLETE Column stats: NONE
+                    Group By Operator
+                      aggregations: count(_col3)
+                      keys: _col0 (type: char(30)), _col1 (type: char(20)), _col2 (type: date)
+                      minReductionHashAggr: 0.99
+                      mode: hash
+                      outputColumnNames: _col0, _col1, _col2, _col3
+                      Statistics: Num rows: 3 Data size: 846 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: char(30)), _col1 (type: char(20)), _col2 (type: date)
+                        sort order: +++
+                        Map-reduce partition columns: _col0 (type: char(30)), _col1 (type: char(20)), _col2 (type: date)
+                        Statistics: Num rows: 3 Data size: 846 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col3 (type: bigint)
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Map 5 
+            Map Operator Tree:
+                TableScan
+                  alias: catalog_sales
+                  Statistics: Num rows: 1 Data size: 282 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: c_last_name (type: char(30)), c_first_name (type: char(20)), d_date (type: date), $f3 (type: bigint)
+                    outputColumnNames: _col0, _col1, _col2, _col3
+                    Statistics: Num rows: 1 Data size: 282 Basic stats: COMPLETE Column stats: NONE
+                    Group By Operator
+                      aggregations: count(_col3)
+                      keys: _col0 (type: char(30)), _col1 (type: char(20)), _col2 (type: date)
+                      minReductionHashAggr: 0.99
+                      mode: hash
+                      outputColumnNames: _col0, _col1, _col2, _col3
+                      Statistics: Num rows: 3 Data size: 846 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: char(30)), _col1 (type: char(20)), _col2 (type: date)
+                        sort order: +++
+                        Map-reduce partition columns: _col0 (type: char(30)), _col1 (type: char(20)), _col2 (type: date)
+                        Statistics: Num rows: 3 Data size: 846 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col3 (type: bigint)
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Map 6 
+            Map Operator Tree:
+                TableScan
+                  alias: web_sales
+                  Statistics: Num rows: 1 Data size: 282 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: c_last_name (type: char(30)), c_first_name (type: char(20)), d_date (type: date), $f3 (type: bigint)
+                    outputColumnNames: _col0, _col1, _col2, _col3
+                    Statistics: Num rows: 1 Data size: 282 Basic stats: COMPLETE Column stats: NONE
+                    Group By Operator
+                      aggregations: count(_col3)
+                      keys: _col0 (type: char(30)), _col1 (type: char(20)), _col2 (type: date)
+                      minReductionHashAggr: 0.99
+                      mode: hash
+                      outputColumnNames: _col0, _col1, _col2, _col3
+                      Statistics: Num rows: 3 Data size: 846 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: char(30)), _col1 (type: char(20)), _col2 (type: date)
+                        sort order: +++
+                        Map-reduce partition columns: _col0 (type: char(30)), _col1 (type: char(20)), _col2 (type: date)
+                        Statistics: Num rows: 3 Data size: 846 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col3 (type: bigint)
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                keys: KEY._col0 (type: char(30)), KEY._col1 (type: char(20)), KEY._col2 (type: date)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1, _col2, _col3
+                Statistics: Num rows: 1 Data size: 282 Basic stats: COMPLETE Column stats: NONE
+                Select Operator
+                  expressions: _col3 (type: bigint)
+                  outputColumnNames: _col3
+                  Statistics: Num rows: 1 Data size: 282 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (_col3 = 3L) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 282 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      Statistics: Num rows: 1 Data size: 282 Basic stats: COMPLETE Column stats: NONE
+                      Group By Operator
+                        aggregations: count()
+                        minReductionHashAggr: 0.99
+                        mode: hash
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 1 Data size: 298 Basic stats: COMPLETE Column stats: NONE
+                        Reduce Output Operator
+                          sort order: 
+                          Statistics: Num rows: 1 Data size: 298 Basic stats: COMPLETE Column stats: NONE
+                          value expressions: _col0 (type: bigint)
+        Reducer 4 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 298 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 298 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Union 2 
+            Vertex: Union 2
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT Count(*) 
+FROM   (SELECT DISTINCT c_last_name, 
+                        c_first_name, 
+                        d_date 
+        FROM   store_sales, 
+               date_dim, 
+               customer 
+        WHERE  store_sales.ss_sold_date_sk = date_dim.d_date_sk 
+               AND store_sales.ss_customer_sk = customer.c_customer_sk 
+               AND d_month_seq BETWEEN 1212 AND 1212 + 11 
+        intersect 
+        SELECT DISTINCT c_last_name, 
+                        c_first_name, 
+                        d_date 
+        FROM   catalog_sales, 
+               date_dim, 
+               customer 
+        WHERE  catalog_sales.cs_sold_date_sk = date_dim.d_date_sk 
+               AND catalog_sales.cs_bill_customer_sk = customer.c_customer_sk 
+               AND d_month_seq BETWEEN 1212 AND 1212 + 11 
+        intersect 
+        SELECT DISTINCT c_last_name, 
+                        c_first_name, 
+                        d_date 
+        FROM   web_sales, 
+               date_dim, 
+               customer 
+        WHERE  web_sales.ws_sold_date_sk = date_dim.d_date_sk 
+               AND web_sales.ws_bill_customer_sk = customer.c_customer_sk 
+               AND d_month_seq BETWEEN 1212 AND 1212 + 11) hot_cust 
+LIMIT  100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@catalog_sales
+PREHOOK: Input: default@customer
+PREHOOK: Input: default@date_dim
+PREHOOK: Input: default@store_sales
+PREHOOK: Input: default@web_sales
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT Count(*) 
+FROM   (SELECT DISTINCT c_last_name, 
+                        c_first_name, 
+                        d_date 
+        FROM   store_sales, 
+               date_dim, 
+               customer 
+        WHERE  store_sales.ss_sold_date_sk = date_dim.d_date_sk 
+               AND store_sales.ss_customer_sk = customer.c_customer_sk 
+               AND d_month_seq BETWEEN 1212 AND 1212 + 11 
+        intersect 
+        SELECT DISTINCT c_last_name, 
+                        c_first_name, 
+                        d_date 
+        FROM   catalog_sales, 
+               date_dim, 
+               customer 
+        WHERE  catalog_sales.cs_sold_date_sk = date_dim.d_date_sk 
+               AND catalog_sales.cs_bill_customer_sk = customer.c_customer_sk 
+               AND d_month_seq BETWEEN 1212 AND 1212 + 11 
+        intersect 
+        SELECT DISTINCT c_last_name, 
+                        c_first_name, 
+                        d_date 
+        FROM   web_sales, 
+               date_dim, 
+               customer 
+        WHERE  web_sales.ws_sold_date_sk = date_dim.d_date_sk 
+               AND web_sales.ws_bill_customer_sk = customer.c_customer_sk 
+               AND d_month_seq BETWEEN 1212 AND 1212 + 11) hot_cust 
+LIMIT  100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@catalog_sales
+POSTHOOK: Input: default@customer
+POSTHOOK: Input: default@date_dim
+POSTHOOK: Input: default@store_sales
+POSTHOOK: Input: default@web_sales
+#### A masked pattern was here ####
+0
+PREHOOK: query: explain
+WITH ss AS 
+( 
+         SELECT   i_item_id, 
+                  Sum(ss_ext_sales_price) total_sales 
+         FROM     store_sales, 
+                  date_dim, 
+                  customer_address, 
+                  item 
+         WHERE    i_item_id IN 
+                  ( 
+                         SELECT i_item_id 
+                         FROM   item 
+                         WHERE  i_color IN ('orchid', 
+                                            'chiffon', 
+                                            'lace')) 
+         AND      ss_item_sk = i_item_sk 
+         AND      ss_sold_date_sk = d_date_sk 
+         AND      d_year = 2000 
+         AND      d_moy = 1 
+         AND      ss_addr_sk = ca_address_sk 
+         AND      ca_gmt_offset = -8 
+         GROUP BY i_item_id), cs AS 
+( 
+         SELECT   i_item_id, 
+                  Sum(cs_ext_sales_price) total_sales 
+         FROM     catalog_sales, 
+                  date_dim, 
+                  customer_address, 
+                  item 
+         WHERE    i_item_id IN 
+                  ( 
+                         SELECT i_item_id 
+                         FROM   item 
+                         WHERE  i_color IN ('orchid', 
+                                            'chiffon', 
+                                            'lace')) 
+         AND      cs_item_sk = i_item_sk 
+         AND      cs_sold_date_sk = d_date_sk 
+         AND      d_year = 2000 
+         AND      d_moy = 1 
+         AND      cs_bill_addr_sk = ca_address_sk 
+         AND      ca_gmt_offset = -8 
+         GROUP BY i_item_id), ws AS 
+( 
+         SELECT   i_item_id, 
+                  Sum(ws_ext_sales_price) total_sales 
+         FROM     web_sales, 
+                  date_dim, 
+                  customer_address, 
+                  item 
+         WHERE    i_item_id IN 
+                  ( 
+                         SELECT i_item_id 
+                         FROM   item 
+                         WHERE  i_color IN ('orchid', 
+                                            'chiffon', 
+                                            'lace')) 
+         AND      ws_item_sk = i_item_sk 
+         AND      ws_sold_date_sk = d_date_sk 
+         AND      d_year = 2000 
+         AND      d_moy = 1 
+         AND      ws_bill_addr_sk = ca_address_sk 
+         AND      ca_gmt_offset = -8 
+         GROUP BY i_item_id) 
+SELECT   i_item_id , 
+         Sum(total_sales) total_sales 
+FROM     ( 
+                SELECT * 
+                FROM   ss 
+                UNION ALL 
+                SELECT * 
+                FROM   cs 
+                UNION ALL 
+                SELECT * 
+                FROM   ws) tmp1 
+GROUP BY i_item_id 
+ORDER BY total_sales limit 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@catalog_sales
+PREHOOK: Input: default@customer_address
+PREHOOK: Input: default@date_dim
+PREHOOK: Input: default@item
+PREHOOK: Input: default@store_sales
+PREHOOK: Input: default@web_sales
+#### A masked pattern was here ####
+POSTHOOK: query: explain
+WITH ss AS 
+( 
+         SELECT   i_item_id, 
+                  Sum(ss_ext_sales_price) total_sales 
+         FROM     store_sales, 
+                  date_dim, 
+                  customer_address, 
+                  item 
+         WHERE    i_item_id IN 
+                  ( 
+                         SELECT i_item_id 
+                         FROM   item 
+                         WHERE  i_color IN ('orchid', 
+                                            'chiffon', 
+                                            'lace')) 
+         AND      ss_item_sk = i_item_sk 
+         AND      ss_sold_date_sk = d_date_sk 
+         AND      d_year = 2000 
+         AND      d_moy = 1 
+         AND      ss_addr_sk = ca_address_sk 
+         AND      ca_gmt_offset = -8 
+         GROUP BY i_item_id), cs AS 
+( 
+         SELECT   i_item_id, 
+                  Sum(cs_ext_sales_price) total_sales 
+         FROM     catalog_sales, 
+                  date_dim, 
+                  customer_address, 
+                  item 
+         WHERE    i_item_id IN 
+                  ( 
+                         SELECT i_item_id 
+                         FROM   item 
+                         WHERE  i_color IN ('orchid', 
+                                            'chiffon', 
+                                            'lace')) 
+         AND      cs_item_sk = i_item_sk 
+         AND      cs_sold_date_sk = d_date_sk 
+         AND      d_year = 2000 
+         AND      d_moy = 1 
+         AND      cs_bill_addr_sk = ca_address_sk 
+         AND      ca_gmt_offset = -8 
+         GROUP BY i_item_id), ws AS 
+( 
+         SELECT   i_item_id, 
+                  Sum(ws_ext_sales_price) total_sales 
+         FROM     web_sales, 
+                  date_dim, 
+                  customer_address, 
+                  item 
+         WHERE    i_item_id IN 
+                  ( 
+                         SELECT i_item_id 
+                         FROM   item 
+                         WHERE  i_color IN ('orchid', 
+                                            'chiffon', 
+                                            'lace')) 
+         AND      ws_item_sk = i_item_sk 
+         AND      ws_sold_date_sk = d_date_sk 
+         AND      d_year = 2000 
+         AND      d_moy = 1 
+         AND      ws_bill_addr_sk = ca_address_sk 
+         AND      ca_gmt_offset = -8 
+         GROUP BY i_item_id) 
+SELECT   i_item_id , 
+         Sum(total_sales) total_sales 
+FROM     ( 
+                SELECT * 
+                FROM   ss 
+                UNION ALL 
+                SELECT * 
+                FROM   cs 
+                UNION ALL 
+                SELECT * 
+                FROM   ws) tmp1 
+GROUP BY i_item_id 
+ORDER BY total_sales limit 100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@catalog_sales
+POSTHOOK: Input: default@customer_address
+POSTHOOK: Input: default@date_dim
+POSTHOOK: Input: default@item
+POSTHOOK: Input: default@store_sales
+POSTHOOK: Input: default@web_sales
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 10 <- Map 13 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
+        Reducer 11 <- Reducer 10 (SIMPLE_EDGE), Union 4 (CONTAINS)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Union 4 (CONTAINS)
+        Reducer 5 <- Union 4 (SIMPLE_EDGE)
+        Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
+        Reducer 8 <- Map 12 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
+        Reducer 9 <- Reducer 8 (SIMPLE_EDGE), Union 4 (CONTAINS)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: store_sales
+                  properties:
+                    hive.sql.query SELECT "t0"."ss_sold_date_sk", "t0"."ss_item_sk", "t0"."ss_addr_sk", "t0"."ss_ext_sales_price", "t2"."d_date_sk", "t4"."ca_address_sk", "t6"."i_item_sk", "t6"."i_item_id", "t6"."i_rec_start_date", "t6"."i_rec_end_date", "t6"."i_item_desc", "t6"."i_current_price", "t6"."i_wholesale_cost", "t6"."i_brand_id", "t6"."i_brand", "t6"."i_class_id", "t6"."i_class", "t6"."i_category_id", "t6"."i_category", "t6"."i_manufact_id", "t6"."i_manufact", "t6"."i_size", " [...]
+FROM (SELECT "ss_sold_date_sk", "ss_item_sk", "ss_addr_sk", "ss_ext_sales_price"
+FROM "STORE_SALES"
+WHERE "ss_sold_date_sk" IS NOT NULL AND "ss_addr_sk" IS NOT NULL AND "ss_item_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "d_date_sk"
+FROM "DATE_DIM"
+WHERE "d_year" = 2000 AND "d_moy" = 1 AND "d_date_sk" IS NOT NULL) AS "t2" ON "t0"."ss_sold_date_sk" = "t2"."d_date_sk"
+INNER JOIN (SELECT "ca_address_sk"
+FROM "CUSTOMER_ADDRESS"
+WHERE "ca_gmt_offset" = -8 AND "ca_address_sk" IS NOT NULL) AS "t4" ON "t0"."ss_addr_sk" = "t4"."ca_address_sk"
+INNER JOIN (SELECT "i_item_sk", "i_item_id", "i_rec_start_date", "i_rec_end_date", "i_item_desc", "i_current_price", "i_wholesale_cost", "i_brand_id", "i_brand", "i_class_id", "i_class", "i_category_id", "i_category", "i_manufact_id", "i_manufact", "i_size", "i_formulation", "i_color", "i_units", "i_container", "i_manager_id", "i_product_name"
+FROM "ITEM"
+WHERE "i_item_id" IS NOT NULL AND "i_item_sk" IS NOT NULL) AS "t6" ON "t0"."ss_item_sk" = "t6"."i_item_sk"
+                    hive.sql.query.fieldNames ss_sold_date_sk,ss_item_sk,ss_addr_sk,ss_ext_sales_price,d_date_sk,ca_address_sk,i_item_sk,i_item_id,i_rec_start_date,i_rec_end_date,i_item_desc,i_current_price,i_wholesale_cost,i_brand_id,i_brand,i_class_id,i_class,i_category_id,i_category,i_manufact_id,i_manufact,i_size,i_formulation,i_color,i_units,i_container,i_manager_id,i_product_name
+                    hive.sql.query.fieldTypes int,int,int,decimal(7,2),int,int,int,char(16),date,date,varchar(200),decimal(7,2),decimal(7,2),int,char(50),int,char(50),int,char(50),int,char(50),char(20),char(20),char(20),char(10),char(10),int,char(50)
+                    hive.sql.query.split false
+                  Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: ss_ext_sales_price (type: decimal(7,2)), i_item_id (type: char(16))
+                    outputColumnNames: _col3, _col7
+                    Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: _col7 (type: char(16))
+                      sort order: +
+                      Map-reduce partition columns: _col7 (type: char(16))
+                      Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col3 (type: decimal(7,2))
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Map 12 
+            Map Operator Tree:
+                TableScan
+                  alias: catalog_sales
+                  properties:
+                    hive.sql.query SELECT "t0"."cs_sold_date_sk", "t0"."cs_bill_addr_sk", "t0"."cs_item_sk", "t0"."cs_ext_sales_price", "t2"."d_date_sk", "t4"."ca_address_sk", "t6"."i_item_sk", "t6"."i_item_id", "t6"."i_rec_start_date", "t6"."i_rec_end_date", "t6"."i_item_desc", "t6"."i_current_price", "t6"."i_wholesale_cost", "t6"."i_brand_id", "t6"."i_brand", "t6"."i_class_id", "t6"."i_class", "t6"."i_category_id", "t6"."i_category", "t6"."i_manufact_id", "t6"."i_manufact", "t6"."i_siz [...]
+FROM (SELECT "cs_sold_date_sk", "cs_bill_addr_sk", "cs_item_sk", "cs_ext_sales_price"
+FROM "CATALOG_SALES"
+WHERE "cs_sold_date_sk" IS NOT NULL AND "cs_bill_addr_sk" IS NOT NULL AND "cs_item_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "d_date_sk"
+FROM "DATE_DIM"
+WHERE "d_year" = 2000 AND "d_moy" = 1 AND "d_date_sk" IS NOT NULL) AS "t2" ON "t0"."cs_sold_date_sk" = "t2"."d_date_sk"
+INNER JOIN (SELECT "ca_address_sk"
+FROM "CUSTOMER_ADDRESS"
+WHERE "ca_gmt_offset" = -8 AND "ca_address_sk" IS NOT NULL) AS "t4" ON "t0"."cs_bill_addr_sk" = "t4"."ca_address_sk"
+INNER JOIN (SELECT "i_item_sk", "i_item_id", "i_rec_start_date", "i_rec_end_date", "i_item_desc", "i_current_price", "i_wholesale_cost", "i_brand_id", "i_brand", "i_class_id", "i_class", "i_category_id", "i_category", "i_manufact_id", "i_manufact", "i_size", "i_formulation", "i_color", "i_units", "i_container", "i_manager_id", "i_product_name"
+FROM "ITEM"
+WHERE "i_item_id" IS NOT NULL AND "i_item_sk" IS NOT NULL) AS "t6" ON "t0"."cs_item_sk" = "t6"."i_item_sk"
+                    hive.sql.query.fieldNames cs_sold_date_sk,cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,d_date_sk,ca_address_sk,i_item_sk,i_item_id,i_rec_start_date,i_rec_end_date,i_item_desc,i_current_price,i_wholesale_cost,i_brand_id,i_brand,i_class_id,i_class,i_category_id,i_category,i_manufact_id,i_manufact,i_size,i_formulation,i_color,i_units,i_container,i_manager_id,i_product_name
+                    hive.sql.query.fieldTypes int,int,int,decimal(7,2),int,int,int,char(16),date,date,varchar(200),decimal(7,2),decimal(7,2),int,char(50),int,char(50),int,char(50),int,char(50),char(20),char(20),char(20),char(10),char(10),int,char(50)
+                    hive.sql.query.split false
+                  Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: cs_ext_sales_price (type: decimal(7,2)), i_item_id (type: char(16))
+                    outputColumnNames: _col3, _col7
+                    Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: _col7 (type: char(16))
+                      sort order: +
+                      Map-reduce partition columns: _col7 (type: char(16))
+                      Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col3 (type: decimal(7,2))
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Map 13 
+            Map Operator Tree:
+                TableScan
+                  alias: web_sales
+                  properties:
+                    hive.sql.query SELECT "t0"."ws_sold_date_sk", "t0"."ws_item_sk", "t0"."ws_bill_addr_sk", "t0"."ws_ext_sales_price", "t2"."d_date_sk", "t4"."ca_address_sk", "t6"."i_item_sk", "t6"."i_item_id", "t6"."i_rec_start_date", "t6"."i_rec_end_date", "t6"."i_item_desc", "t6"."i_current_price", "t6"."i_wholesale_cost", "t6"."i_brand_id", "t6"."i_brand", "t6"."i_class_id", "t6"."i_class", "t6"."i_category_id", "t6"."i_category", "t6"."i_manufact_id", "t6"."i_manufact", "t6"."i_siz [...]
+FROM (SELECT "ws_sold_date_sk", "ws_item_sk", "ws_bill_addr_sk", "ws_ext_sales_price"
+FROM "WEB_SALES"
+WHERE "ws_sold_date_sk" IS NOT NULL AND "ws_bill_addr_sk" IS NOT NULL AND "ws_item_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "d_date_sk"
+FROM "DATE_DIM"
+WHERE "d_year" = 2000 AND "d_moy" = 1 AND "d_date_sk" IS NOT NULL) AS "t2" ON "t0"."ws_sold_date_sk" = "t2"."d_date_sk"
+INNER JOIN (SELECT "ca_address_sk"
+FROM "CUSTOMER_ADDRESS"
+WHERE "ca_gmt_offset" = -8 AND "ca_address_sk" IS NOT NULL) AS "t4" ON "t0"."ws_bill_addr_sk" = "t4"."ca_address_sk"
+INNER JOIN (SELECT "i_item_sk", "i_item_id", "i_rec_start_date", "i_rec_end_date", "i_item_desc", "i_current_price", "i_wholesale_cost", "i_brand_id", "i_brand", "i_class_id", "i_class", "i_category_id", "i_category", "i_manufact_id", "i_manufact", "i_size", "i_formulation", "i_color", "i_units", "i_container", "i_manager_id", "i_product_name"
+FROM "ITEM"
+WHERE "i_item_id" IS NOT NULL AND "i_item_sk" IS NOT NULL) AS "t6" ON "t0"."ws_item_sk" = "t6"."i_item_sk"
+                    hive.sql.query.fieldNames ws_sold_date_sk,ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,d_date_sk,ca_address_sk,i_item_sk,i_item_id,i_rec_start_date,i_rec_end_date,i_item_desc,i_current_price,i_wholesale_cost,i_brand_id,i_brand,i_class_id,i_class,i_category_id,i_category,i_manufact_id,i_manufact,i_size,i_formulation,i_color,i_units,i_container,i_manager_id,i_product_name
+                    hive.sql.query.fieldTypes int,int,int,decimal(7,2),int,int,int,char(16),date,date,varchar(200),decimal(7,2),decimal(7,2),int,char(50),int,char(50),int,char(50),int,char(50),char(20),char(20),char(20),char(10),char(10),int,char(50)
+                    hive.sql.query.split false
+                  Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: ws_ext_sales_price (type: decimal(7,2)), i_item_id (type: char(16))
+                    outputColumnNames: _col3, _col7
+                    Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: _col7 (type: char(16))
+                      sort order: +
+                      Map-reduce partition columns: _col7 (type: char(16))
+                      Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col3 (type: decimal(7,2))
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Map 7 
+            Map Operator Tree:
+                TableScan
+                  alias: item
+                  properties:
+                    hive.sql.query SELECT "i_item_id"
+FROM "ITEM"
+WHERE "i_color" IN ('orchid              ', 'chiffon             ', 'lace                ') AND "i_item_id" IS NOT NULL
+                    hive.sql.query.fieldNames i_item_id
+                    hive.sql.query.fieldTypes char(16)
+                    hive.sql.query.split true
+                  Statistics: Num rows: 1 Data size: 100 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: i_item_id (type: char(16))
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 100 Basic stats: COMPLETE Column stats: NONE
+                    Group By Operator
+                      keys: _col0 (type: char(16))
+                      minReductionHashAggr: 0.99
+                      mode: hash
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 100 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: char(16))
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: char(16))
+                        Statistics: Num rows: 1 Data size: 100 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: char(16))
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: char(16))
+                        Statistics: Num rows: 1 Data size: 100 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: char(16))
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: char(16))
+                        Statistics: Num rows: 1 Data size: 100 Basic stats: COMPLETE Column stats: NONE
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Reducer 10 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Left Semi Join 0 to 1
+                keys:
+                  0 _col7 (type: char(16))
+                  1 _col0 (type: char(16))
+                outputColumnNames: _col3, _col7
+                Statistics: Num rows: 1 Data size: 233 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: sum(_col3)
+                  keys: _col7 (type: char(16))
+                  minReductionHashAggr: 0.99
+                  mode: hash
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 1 Data size: 233 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: char(16))
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: char(16))
+                    Statistics: Num rows: 1 Data size: 233 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col1 (type: decimal(17,2))
+        Reducer 11 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: sum(VALUE._col0)
+                keys: KEY._col0 (type: char(16))
+                mode: mergepartial
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 233 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: sum(_col1)
+                  keys: _col0 (type: char(16))
+                  minReductionHashAggr: 0.99
+                  mode: hash
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 3 Data size: 699 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: char(16))
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: char(16))
+                    Statistics: Num rows: 3 Data size: 699 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col1 (type: decimal(27,2))
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Left Semi Join 0 to 1
+                keys:
+                  0 _col7 (type: char(16))
+                  1 _col0 (type: char(16))
+                outputColumnNames: _col3, _col7
+                Statistics: Num rows: 1 Data size: 233 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: sum(_col3)
+                  keys: _col7 (type: char(16))
+                  minReductionHashAggr: 0.99
+                  mode: hash
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 1 Data size: 233 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: char(16))
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: char(16))
+                    Statistics: Num rows: 1 Data size: 233 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col1 (type: decimal(17,2))
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: sum(VALUE._col0)
+                keys: KEY._col0 (type: char(16))
+                mode: mergepartial
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 233 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: sum(_col1)
+                  keys: _col0 (type: char(16))
+                  minReductionHashAggr: 0.99
+                  mode: hash
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 3 Data size: 699 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: char(16))
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: char(16))
+                    Statistics: Num rows: 3 Data size: 699 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col1 (type: decimal(27,2))
+        Reducer 5 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: sum(VALUE._col0)
+                keys: KEY._col0 (type: char(16))
+                mode: mergepartial
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 233 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col1 (type: decimal(27,2))
+                  sort order: +
+                  Statistics: Num rows: 1 Data size: 233 Basic stats: COMPLETE Column stats: NONE
+                  TopN Hash Memory Usage: 0.1
+                  value expressions: _col0 (type: char(16))
+        Reducer 6 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: VALUE._col0 (type: char(16)), KEY.reducesinkkey0 (type: decimal(27,2))
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 233 Basic stats: COMPLETE Column stats: NONE
+                Limit
+                  Number of rows: 100
+                  Statistics: Num rows: 1 Data size: 233 Basic stats: COMPLETE Column stats: NONE
+                  File Output Operator
+                    compressed: false
+                    Statistics: Num rows: 1 Data size: 233 Basic stats: COMPLETE Column stats: NONE
+                    table:
+                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 8 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Left Semi Join 0 to 1
+                keys:
+                  0 _col7 (type: char(16))
+                  1 _col0 (type: char(16))
+                outputColumnNames: _col3, _col7
+                Statistics: Num rows: 1 Data size: 233 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: sum(_col3)
+                  keys: _col7 (type: char(16))
+                  minReductionHashAggr: 0.99
+                  mode: hash
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 1 Data size: 233 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: char(16))
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: char(16))
+                    Statistics: Num rows: 1 Data size: 233 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col1 (type: decimal(17,2))
+        Reducer 9 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: sum(VALUE._col0)
+                keys: KEY._col0 (type: char(16))
+                mode: mergepartial
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 233 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: sum(_col1)
+                  keys: _col0 (type: char(16))
+                  minReductionHashAggr: 0.99
+                  mode: hash
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 3 Data size: 699 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: char(16))
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: char(16))
+                    Statistics: Num rows: 3 Data size: 699 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col1 (type: decimal(27,2))
+        Union 4 
+            Vertex: Union 4
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: 100
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: WITH ss AS 
+( 
+         SELECT   i_item_id, 
+                  Sum(ss_ext_sales_price) total_sales 
+         FROM     store_sales, 
+                  date_dim, 
+                  customer_address, 
+                  item 
+         WHERE    i_item_id IN 
+                  ( 
+                         SELECT i_item_id 
+                         FROM   item 
+                         WHERE  i_color IN ('orchid', 
+                                            'chiffon', 
+                                            'lace')) 
+         AND      ss_item_sk = i_item_sk 
+         AND      ss_sold_date_sk = d_date_sk 
+         AND      d_year = 2000 
+         AND      d_moy = 1 
+         AND      ss_addr_sk = ca_address_sk 
+         AND      ca_gmt_offset = -8 
+         GROUP BY i_item_id), cs AS 
+( 
+         SELECT   i_item_id, 
+                  Sum(cs_ext_sales_price) total_sales 
+         FROM     catalog_sales, 
+                  date_dim, 
+                  customer_address, 
+                  item 
+         WHERE    i_item_id IN 
+                  ( 
+                         SELECT i_item_id 
+                         FROM   item 
+                         WHERE  i_color IN ('orchid', 
+                                            'chiffon', 
+                                            'lace')) 
+         AND      cs_item_sk = i_item_sk 
+         AND      cs_sold_date_sk = d_date_sk 
+         AND      d_year = 2000 
+         AND      d_moy = 1 
+         AND      cs_bill_addr_sk = ca_address_sk 
+         AND      ca_gmt_offset = -8 
+         GROUP BY i_item_id), ws AS 
+( 
+         SELECT   i_item_id, 
+                  Sum(ws_ext_sales_price) total_sales 
+         FROM     web_sales, 
+                  date_dim, 
+                  customer_address, 
+                  item 
+         WHERE    i_item_id IN 
+                  ( 
+                         SELECT i_item_id 
+                         FROM   item 
+                         WHERE  i_color IN ('orchid', 
+                                            'chiffon', 
+                                            'lace')) 
+         AND      ws_item_sk = i_item_sk 
+         AND      ws_sold_date_sk = d_date_sk 
+         AND      d_year = 2000 
+         AND      d_moy = 1 
+         AND      ws_bill_addr_sk = ca_address_sk 
+         AND      ca_gmt_offset = -8 
+         GROUP BY i_item_id) 
+SELECT   i_item_id , 
+         Sum(total_sales) total_sales 
+FROM     ( 
+                SELECT * 
+                FROM   ss 
+                UNION ALL 
+                SELECT * 
+                FROM   cs 
+                UNION ALL 
+                SELECT * 
+                FROM   ws) tmp1 
+GROUP BY i_item_id 
+ORDER BY total_sales limit 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@catalog_sales
+PREHOOK: Input: default@customer_address
+PREHOOK: Input: default@date_dim
+PREHOOK: Input: default@item
+PREHOOK: Input: default@store_sales
+PREHOOK: Input: default@web_sales
+#### A masked pattern was here ####
+POSTHOOK: query: WITH ss AS 
+( 
+         SELECT   i_item_id, 
+                  Sum(ss_ext_sales_price) total_sales 
+         FROM     store_sales, 
+                  date_dim, 
+                  customer_address, 
+                  item 
+         WHERE    i_item_id IN 
+                  ( 
+                         SELECT i_item_id 
+                         FROM   item 
+                         WHERE  i_color IN ('orchid', 
+                                            'chiffon', 
+                                            'lace')) 
+         AND      ss_item_sk = i_item_sk 
+         AND      ss_sold_date_sk = d_date_sk 
+         AND      d_year = 2000 
+         AND      d_moy = 1 
+         AND      ss_addr_sk = ca_address_sk 
+         AND      ca_gmt_offset = -8 
+         GROUP BY i_item_id), cs AS 
+( 
+         SELECT   i_item_id, 
+                  Sum(cs_ext_sales_price) total_sales 
+         FROM     catalog_sales, 
+                  date_dim, 
+                  customer_address, 
+                  item 
+         WHERE    i_item_id IN 
+                  ( 
+                         SELECT i_item_id 
+                         FROM   item 
+                         WHERE  i_color IN ('orchid', 
+                                            'chiffon', 
+                                            'lace')) 
+         AND      cs_item_sk = i_item_sk 
+         AND      cs_sold_date_sk = d_date_sk 
+         AND      d_year = 2000 
+         AND      d_moy = 1 
+         AND      cs_bill_addr_sk = ca_address_sk 
+         AND      ca_gmt_offset = -8 
+         GROUP BY i_item_id), ws AS 
+( 
+         SELECT   i_item_id, 
+                  Sum(ws_ext_sales_price) total_sales 
+         FROM     web_sales, 
+                  date_dim, 
+                  customer_address, 
+                  item 
+         WHERE    i_item_id IN 
+                  ( 
+                         SELECT i_item_id 
+                         FROM   item 
+                         WHERE  i_color IN ('orchid', 
+                                            'chiffon', 
+                                            'lace')) 
+         AND      ws_item_sk = i_item_sk 
+         AND      ws_sold_date_sk = d_date_sk 
+         AND      d_year = 2000 
+         AND      d_moy = 1 
+         AND      ws_bill_addr_sk = ca_address_sk 
+         AND      ca_gmt_offset = -8 
+         GROUP BY i_item_id) 
+SELECT   i_item_id , 
+         Sum(total_sales) total_sales 
+FROM     ( 
+                SELECT * 
+                FROM   ss 
+                UNION ALL 
+                SELECT * 
+                FROM   cs 
+                UNION ALL 
+                SELECT * 
+                FROM   ws) tmp1 
+GROUP BY i_item_id 
+ORDER BY total_sales limit 100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@catalog_sales
+POSTHOOK: Input: default@customer_address
+POSTHOOK: Input: default@date_dim
+POSTHOOK: Input: default@item
+POSTHOOK: Input: default@store_sales
+POSTHOOK: Input: default@web_sales
+#### A masked pattern was here ####
+PREHOOK: query: explain
+WITH sr_items AS 
+( 
+         SELECT   i_item_id               item_id, 
+                  Sum(sr_return_quantity) sr_item_qty 
+         FROM     store_returns, 
+                  item, 
+                  date_dim 
+         WHERE    sr_item_sk = i_item_sk 
+         AND      d_date IN 
+                  ( 
+                         SELECT d_date 
+                         FROM   date_dim 
+                         WHERE  d_week_seq IN 
+                                ( 
+                                       SELECT d_week_seq 
+                                       FROM   date_dim 
+                                       WHERE  d_date IN ('1998-01-02', 
+                                                         '1998-10-15', 
+                                                         '1998-11-10'))) 
+         AND      sr_returned_date_sk = d_date_sk 
+         GROUP BY i_item_id), cr_items AS 
+( 
+         SELECT   i_item_id               item_id, 
+                  Sum(cr_return_quantity) cr_item_qty 
+         FROM     catalog_returns, 
+                  item, 
+                  date_dim 
+         WHERE    cr_item_sk = i_item_sk 
+         AND      d_date IN 
+                  ( 
+                         SELECT d_date 
+                         FROM   date_dim 
+                         WHERE  d_week_seq IN 
+                                ( 
+                                       SELECT d_week_seq 
+                                       FROM   date_dim 
+                                       WHERE  d_date IN ('1998-01-02', 
+                                                         '1998-10-15', 
+                                                         '1998-11-10'))) 
+         AND      cr_returned_date_sk = d_date_sk 
+         GROUP BY i_item_id), wr_items AS 
+( 
+         SELECT   i_item_id               item_id, 
+                  Sum(wr_return_quantity) wr_item_qty 
+         FROM     web_returns, 
+                  item, 
+                  date_dim 
+         WHERE    wr_item_sk = i_item_sk 
+         AND      d_date IN 
+                  ( 
+                         SELECT d_date 
+                         FROM   date_dim 
+                         WHERE  d_week_seq IN 
+                                ( 
+                                       SELECT d_week_seq 
+                                       FROM   date_dim 
+                                       WHERE  d_date IN ('1998-01-02', 
+                                                         '1998-10-15', 
+                                                         '1998-11-10'))) 
+         AND      wr_returned_date_sk = d_date_sk 
+         GROUP BY i_item_id) 
+SELECT   sr_items.item_id , 
+         sr_item_qty , 
+         sr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 sr_dev , 
+         cr_item_qty , 
+         cr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 cr_dev , 
+         wr_item_qty , 
+         wr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 wr_dev , 
+         (sr_item_qty+cr_item_qty+wr_item_qty)/3.0                   average 
+FROM     sr_items , 
+         cr_items , 
+         wr_items 
+WHERE    sr_items.item_id=cr_items.item_id 
+AND      sr_items.item_id=wr_items.item_id 
+ORDER BY sr_items.item_id , 
+         sr_item_qty limit 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@catalog_returns
+PREHOOK: Input: default@date_dim
+PREHOOK: Input: default@item
+PREHOOK: Input: default@store_returns
+PREHOOK: Input: default@web_returns
+#### A masked pattern was here ####
+POSTHOOK: query: explain
+WITH sr_items AS 
+( 
+         SELECT   i_item_id               item_id, 
+                  Sum(sr_return_quantity) sr_item_qty 
+         FROM     store_returns, 
+                  item, 
+                  date_dim 
+         WHERE    sr_item_sk = i_item_sk 
+         AND      d_date IN 
+                  ( 
+                         SELECT d_date 
+                         FROM   date_dim 
+                         WHERE  d_week_seq IN 
+                                ( 
+                                       SELECT d_week_seq 
+                                       FROM   date_dim 
+                                       WHERE  d_date IN ('1998-01-02', 
+                                                         '1998-10-15', 
+                                                         '1998-11-10'))) 
+         AND      sr_returned_date_sk = d_date_sk 
+         GROUP BY i_item_id), cr_items AS 
+( 
+         SELECT   i_item_id               item_id, 
+                  Sum(cr_return_quantity) cr_item_qty 
+         FROM     catalog_returns, 
+                  item, 
+                  date_dim 
+         WHERE    cr_item_sk = i_item_sk 
+         AND      d_date IN 
+                  ( 
+                         SELECT d_date 
+                         FROM   date_dim 
+                         WHERE  d_week_seq IN 
+                                ( 
+                                       SELECT d_week_seq 
+                                       FROM   date_dim 
+                                       WHERE  d_date IN ('1998-01-02', 
+                                                         '1998-10-15', 
+                                                         '1998-11-10'))) 
+         AND      cr_returned_date_sk = d_date_sk 
+         GROUP BY i_item_id), wr_items AS 
+( 
+         SELECT   i_item_id               item_id, 
+                  Sum(wr_return_quantity) wr_item_qty 
+         FROM     web_returns, 
+                  item, 
+                  date_dim 
+         WHERE    wr_item_sk = i_item_sk 
+         AND      d_date IN 
+                  ( 
+                         SELECT d_date 
+                         FROM   date_dim 
+                         WHERE  d_week_seq IN 
+                                ( 
+                                       SELECT d_week_seq 
+                                       FROM   date_dim 
+                                       WHERE  d_date IN ('1998-01-02', 
+                                                         '1998-10-15', 
+                                                         '1998-11-10'))) 
+         AND      wr_returned_date_sk = d_date_sk 
+         GROUP BY i_item_id) 
+SELECT   sr_items.item_id , 
+         sr_item_qty , 
+         sr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 sr_dev , 
+         cr_item_qty , 
+         cr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 cr_dev , 
+         wr_item_qty , 
+         wr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 wr_dev , 
+         (sr_item_qty+cr_item_qty+wr_item_qty)/3.0                   average 
+FROM     sr_items , 
+         cr_items , 
+         wr_items 
+WHERE    sr_items.item_id=cr_items.item_id 
+AND      sr_items.item_id=wr_items.item_id 
+ORDER BY sr_items.item_id , 
+         sr_item_qty limit 100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@catalog_returns
+POSTHOOK: Input: default@date_dim
+POSTHOOK: Input: default@item
+POSTHOOK: Input: default@store_returns
+POSTHOOK: Input: default@web_returns
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 10 <- Map 13 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
+        Reducer 11 <- Reducer 10 (SIMPLE_EDGE)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+        Reducer 4 <- Reducer 3 (ONE_TO_ONE_EDGE), Reducer 9 (ONE_TO_ONE_EDGE)
+        Reducer 5 <- Reducer 11 (ONE_TO_ONE_EDGE), Reducer 4 (ONE_TO_ONE_EDGE)
+        Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
+        Reducer 8 <- Map 12 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
+        Reducer 9 <- Reducer 8 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: store_returns
+                  properties:
+                    hive.sql.query SELECT "t0"."sr_returned_date_sk", "t0"."sr_return_time_sk", "t0"."sr_item_sk", "t0"."sr_customer_sk", "t0"."sr_cdemo_sk", "t0"."sr_hdemo_sk", "t0"."sr_addr_sk", "t0"."sr_store_sk", "t0"."sr_reason_sk", "t0"."sr_ticket_number", "t0"."sr_return_quantity", "t0"."sr_return_amt", "t0"."sr_return_tax", "t0"."sr_return_amt_inc_tax", "t0"."sr_fee", "t0"."sr_return_ship_cost", "t0"."sr_refunded_cash", "t0"."sr_reversed_charge", "t0"."sr_store_credit", "t0"."sr_ [...]
+FROM (SELECT "sr_returned_date_sk", "sr_return_time_sk", "sr_item_sk", "sr_customer_sk", "sr_cdemo_sk", "sr_hdemo_sk", "sr_addr_sk", "sr_store_sk", "sr_reason_sk", "sr_ticket_number", "sr_return_quantity", "sr_return_amt", "sr_return_tax", "sr_return_amt_inc_tax", "sr_fee", "sr_return_ship_cost", "sr_refunded_cash", "sr_reversed_charge", "sr_store_credit", "sr_net_loss"
+FROM "STORE_RETURNS"
+WHERE "sr_item_sk" IS NOT NULL AND "sr_returned_date_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "i_item_sk", "i_item_id", "i_rec_start_date", "i_rec_end_date", "i_item_desc", "i_current_price", "i_wholesale_cost", "i_brand_id", "i_brand", "i_class_id", "i_class", "i_category_id", "i_category", "i_manufact_id", "i_manufact", "i_size", "i_formulation", "i_color", "i_units", "i_container", "i_manager_id", "i_product_name"
+FROM "ITEM"
+WHERE "i_item_sk" IS NOT NULL AND "i_item_id" IS NOT NULL) AS "t2" ON "t0"."sr_item_sk" = "t2"."i_item_sk"
+INNER JOIN (SELECT "d_date_sk", "d_date_id", "d_date", "d_month_seq", "d_week_seq", "d_quarter_seq", "d_year", "d_dow", "d_moy", "d_dom", "d_qoy", "d_fy_year", "d_fy_quarter_seq", "d_fy_week_seq", "d_day_name", "d_quarter_name", "d_holiday", "d_weekend", "d_following_holiday", "d_first_dom", "d_last_dom", "d_same_day_ly", "d_same_day_lq", "d_current_day", "d_current_week", "d_current_month", "d_current_quarter", "d_current_year"
+FROM "DATE_DIM"
+WHERE "d_date" IS NOT NULL AND "d_date_sk" IS NOT NULL) AS "t4" ON "t0"."sr_returned_date_sk" = "t4"."d_date_sk"
+                    hive.sql.query.fieldNames sr_returned_date_sk,sr_return_time_sk,sr_item_sk,sr_customer_sk,sr_cdemo_sk,sr_hdemo_sk,sr_addr_sk,sr_store_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_return_tax,sr_return_amt_inc_tax,sr_fee,sr_return_ship_cost,sr_refunded_cash,sr_reversed_charge,sr_store_credit,sr_net_loss,i_item_sk,i_item_id,i_rec_start_date,i_rec_end_date,i_item_desc,i_current_price,i_wholesale_cost,i_brand_id,i_brand,i_class_id,i_class,i_category [...]
+                    hive.sql.query.fieldTypes int,int,int,int,int,int,int,int,int,int,int,decimal(7,2),decimal(7,2),decimal(7,2),decimal(7,2),decimal(7,2),decimal(7,2),decimal(7,2),decimal(7,2),decimal(7,2),int,char(16),date,date,varchar(200),decimal(7,2),decimal(7,2),int,char(50),int,char(50),int,char(50),int,char(50),char(20),char(20),char(20),char(10),char(10),int,char(50),int,char(16),date,int,int,int,int,int,int,int,int,int,int,int,char(9),char(6),char(1),char(1),char(1),int,int,int [...]
+                    hive.sql.query.split false
+                  Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: sr_return_quantity (type: int), i_item_id (type: char(16)), d_date (type: date)
+                    outputColumnNames: _col10, _col21, _col44
+                    Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: _col44 (type: date)
+                      sort order: +
+                      Map-reduce partition columns: _col44 (type: date)
+                      Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col10 (type: int), _col21 (type: char(16))
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Map 12 
+            Map Operator Tree:
+                TableScan
+                  alias: catalog_returns
+                  properties:
+                    hive.sql.query SELECT "t0"."cr_returned_date_sk", "t0"."cr_returned_time_sk", "t0"."cr_item_sk", "t0"."cr_refunded_customer_sk", "t0"."cr_refunded_cdemo_sk", "t0"."cr_refunded_hdemo_sk", "t0"."cr_refunded_addr_sk", "t0"."cr_returning_customer_sk", "t0"."cr_returning_cdemo_sk", "t0"."cr_returning_hdemo_sk", "t0"."cr_returning_addr_sk", "t0"."cr_call_center_sk", "t0"."cr_catalog_page_sk", "t0"."cr_ship_mode_sk", "t0"."cr_warehouse_sk", "t0"."cr_reason_sk", "t0"."cr_orde [...]
+FROM (SELECT "cr_returned_date_sk", "cr_returned_time_sk", "cr_item_sk", "cr_refunded_customer_sk", "cr_refunded_cdemo_sk", "cr_refunded_hdemo_sk", "cr_refunded_addr_sk", "cr_returning_customer_sk", "cr_returning_cdemo_sk", "cr_returning_hdemo_sk", "cr_returning_addr_sk", "cr_call_center_sk", "cr_catalog_page_sk", "cr_ship_mode_sk", "cr_warehouse_sk", "cr_reason_sk", "cr_order_number", "cr_return_quantity", "cr_return_amount", "cr_return_tax", "cr_return_amt_inc_tax", "cr_fee", "cr_retur [...]
+FROM "CATALOG_RETURNS"
+WHERE "cr_item_sk" IS NOT NULL AND "cr_returned_date_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "i_item_sk", "i_item_id", "i_rec_start_date", "i_rec_end_date", "i_item_desc", "i_current_price", "i_wholesale_cost", "i_brand_id", "i_brand", "i_class_id", "i_class", "i_category_id", "i_category", "i_manufact_id", "i_manufact", "i_size", "i_formulation", "i_color", "i_units", "i_container", "i_manager_id", "i_product_name"
+FROM "ITEM"
+WHERE "i_item_sk" IS NOT NULL AND "i_item_id" IS NOT NULL) AS "t2" ON "t0"."cr_item_sk" = "t2"."i_item_sk"
+INNER JOIN (SELECT "d_date_sk", "d_date_id", "d_date", "d_month_seq", "d_week_seq", "d_quarter_seq", "d_year", "d_dow", "d_moy", "d_dom", "d_qoy", "d_fy_year", "d_fy_quarter_seq", "d_fy_week_seq", "d_day_name", "d_quarter_name", "d_holiday", "d_weekend", "d_following_holiday", "d_first_dom", "d_last_dom", "d_same_day_ly", "d_same_day_lq", "d_current_day", "d_current_week", "d_current_month", "d_current_quarter", "d_current_year"
+FROM "DATE_DIM"
+WHERE "d_date" IS NOT NULL AND "d_date_sk" IS NOT NULL) AS "t4" ON "t0"."cr_returned_date_sk" = "t4"."d_date_sk"
+                    hive.sql.query.fieldNames cr_returned_date_sk,cr_returned_time_sk,cr_item_sk,cr_refunded_customer_sk,cr_refunded_cdemo_sk,cr_refunded_hdemo_sk,cr_refunded_addr_sk,cr_returning_customer_sk,cr_returning_cdemo_sk,cr_returning_hdemo_sk,cr_returning_addr_sk,cr_call_center_sk,cr_catalog_page_sk,cr_ship_mode_sk,cr_warehouse_sk,cr_reason_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_return_tax,cr_return_amt_inc_tax,cr_fee,cr_return_ship_cost,cr_refunded_cash,cr_re [...]
+                    hive.sql.query.fieldTypes int,int,int,int,int,int,int,int,int,int,int,int,int,int,int,int,int,int,decimal(7,2),decimal(7,2),decimal(7,2),decimal(7,2),decimal(7,2),decimal(7,2),decimal(7,2),decimal(7,2),decimal(7,2),int,char(16),date,date,varchar(200),decimal(7,2),decimal(7,2),int,char(50),int,char(50),int,char(50),int,char(50),char(20),char(20),char(20),char(10),char(10),int,char(50),int,char(16),date,int,int,int,int,int,int,int,int,int,int,int,char(9),char(6),char(1) [...]
+                    hive.sql.query.split false
+                  Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: cr_return_quantity (type: int), i_item_id (type: char(16)), d_date (type: date)
+                    outputColumnNames: _col17, _col28, _col51
+                    Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: _col51 (type: date)
+                      sort order: +
+                      Map-reduce partition columns: _col51 (type: date)
+                      Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col17 (type: int), _col28 (type: char(16))
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Map 13 
+            Map Operator Tree:
+                TableScan
+                  alias: web_returns
+                  properties:
+                    hive.sql.query SELECT "t0"."wr_returned_date_sk", "t0"."wr_returned_time_sk", "t0"."wr_item_sk", "t0"."wr_refunded_customer_sk", "t0"."wr_refunded_cdemo_sk", "t0"."wr_refunded_hdemo_sk", "t0"."wr_refunded_addr_sk", "t0"."wr_returning_customer_sk", "t0"."wr_returning_cdemo_sk", "t0"."wr_returning_hdemo_sk", "t0"."wr_returning_addr_sk", "t0"."wr_web_page_sk", "t0"."wr_reason_sk", "t0"."wr_order_number", "t0"."wr_return_quantity", "t0"."wr_return_amt", "t0"."wr_return_ta [...]
+FROM (SELECT "wr_returned_date_sk", "wr_returned_time_sk", "wr_item_sk", "wr_refunded_customer_sk", "wr_refunded_cdemo_sk", "wr_refunded_hdemo_sk", "wr_refunded_addr_sk", "wr_returning_customer_sk", "wr_returning_cdemo_sk", "wr_returning_hdemo_sk", "wr_returning_addr_sk", "wr_web_page_sk", "wr_reason_sk", "wr_order_number", "wr_return_quantity", "wr_return_amt", "wr_return_tax", "wr_return_amt_inc_tax", "wr_fee", "wr_return_ship_cost", "wr_refunded_cash", "wr_reversed_charge", "wr_accoun [...]
+FROM "WEB_RETURNS"
+WHERE "wr_item_sk" IS NOT NULL AND "wr_returned_date_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "i_item_sk", "i_item_id", "i_rec_start_date", "i_rec_end_date", "i_item_desc", "i_current_price", "i_wholesale_cost", "i_brand_id", "i_brand", "i_class_id", "i_class", "i_category_id", "i_category", "i_manufact_id", "i_manufact", "i_size", "i_formulation", "i_color", "i_units", "i_container", "i_manager_id", "i_product_name"
+FROM "ITEM"
+WHERE "i_item_sk" IS NOT NULL AND "i_item_id" IS NOT NULL) AS "t2" ON "t0"."wr_item_sk" = "t2"."i_item_sk"
+INNER JOIN (SELECT "d_date_sk", "d_date_id", "d_date", "d_month_seq", "d_week_seq", "d_quarter_seq", "d_year", "d_dow", "d_moy", "d_dom", "d_qoy", "d_fy_year", "d_fy_quarter_seq", "d_fy_week_seq", "d_day_name", "d_quarter_name", "d_holiday", "d_weekend", "d_following_holiday", "d_first_dom", "d_last_dom", "d_same_day_ly", "d_same_day_lq", "d_current_day", "d_current_week", "d_current_month", "d_current_quarter", "d_current_year"
+FROM "DATE_DIM"
+WHERE "d_date" IS NOT NULL AND "d_date_sk" IS NOT NULL) AS "t4" ON "t0"."wr_returned_date_sk" = "t4"."d_date_sk"
+                    hive.sql.query.fieldNames wr_returned_date_sk,wr_returned_time_sk,wr_item_sk,wr_refunded_customer_sk,wr_refunded_cdemo_sk,wr_refunded_hdemo_sk,wr_refunded_addr_sk,wr_returning_customer_sk,wr_returning_cdemo_sk,wr_returning_hdemo_sk,wr_returning_addr_sk,wr_web_page_sk,wr_reason_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_return_tax,wr_return_amt_inc_tax,wr_fee,wr_return_ship_cost,wr_refunded_cash,wr_reversed_charge,wr_account_credit,wr_net_loss,i_item_sk,i_i [...]
+                    hive.sql.query.fieldTypes int,int,int,int,int,int,int,int,int,int,int,int,int,int,int,decimal(7,2),decimal(7,2),decimal(7,2),decimal(7,2),decimal(7,2),decimal(7,2),decimal(7,2),decimal(7,2),decimal(7,2),int,char(16),date,date,varchar(200),decimal(7,2),decimal(7,2),int,char(50),int,char(50),int,char(50),int,char(50),char(20),char(20),char(20),char(10),char(10),int,char(50),int,char(16),date,int,int,int,int,int,int,int,int,int,int,int,char(9),char(6),char(1),char(1),cha [...]
+                    hive.sql.query.split false
+                  Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: wr_return_quantity (type: int), i_item_id (type: char(16)), d_date (type: date)
+                    outputColumnNames: _col14, _col25, _col48
+                    Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: _col48 (type: date)
+                      sort order: +
+                      Map-reduce partition columns: _col48 (type: date)
+                      Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col14 (type: int), _col25 (type: char(16))
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Map 7 
+            Map Operator Tree:
+                TableScan
+                  alias: date_dim
+                  properties:
+                    hive.sql.query SELECT "t0"."d_date"
+FROM (SELECT "d_date_sk", "d_date_id", "d_date", "d_month_seq", "d_week_seq", "d_quarter_seq", "d_year", "d_dow", "d_moy", "d_dom", "d_qoy", "d_fy_year", "d_fy_quarter_seq", "d_fy_week_seq", "d_day_name", "d_quarter_name", "d_holiday", "d_weekend", "d_following_holiday", "d_first_dom", "d_last_dom", "d_same_day_ly", "d_same_day_lq", "d_current_day", "d_current_week", "d_current_month", "d_current_quarter", "d_current_year"
+FROM "DATE_DIM"
+WHERE "d_week_seq" IS NOT NULL AND "d_date" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "d_week_seq"
+FROM "DATE_DIM"
+WHERE "d_date" IN (DATE '1998-01-02', DATE '1998-10-15', DATE '1998-11-10') AND "d_week_seq" IS NOT NULL
+GROUP BY "d_week_seq") AS "t2" ON "t0"."d_week_seq" = "t2"."d_week_seq"
+                    hive.sql.query.fieldNames d_date
+                    hive.sql.query.fieldTypes date
+                    hive.sql.query.split false
+                  Statistics: Num rows: 1 Data size: 56 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: d_date (type: date)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 56 Basic stats: COMPLETE Column stats: NONE
+                    Group By Operator
+                      keys: _col0 (type: date)
+                      minReductionHashAggr: 0.99
+                      mode: hash
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 56 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: date)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: date)
+                        Statistics: Num rows: 1 Data size: 56 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: date)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: date)
+                        Statistics: Num rows: 1 Data size: 56 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: date)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: date)
+                        Statistics: Num rows: 1 Data size: 56 Basic stats: COMPLETE Column stats: NONE
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Reducer 10 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Left Semi Join 0 to 1
+                keys:
+                  0 _col48 (type: date)
+                  1 _col0 (type: date)
+                outputColumnNames: _col14, _col25
+                Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: sum(_col14)
+                  keys: _col25 (type: char(16))
+                  minReductionHashAggr: 0.99
+                  mode: hash
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: char(16))
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: char(16))
+                    Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col1 (type: bigint)
+        Reducer 11 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: sum(VALUE._col0)
+                keys: KEY._col0 (type: char(16))
+                mode: mergepartial
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: NONE
+                Select Operator
+                  expressions: _col0 (type: char(16)), _col1 (type: bigint), UDFToDouble(_col1) (type: double)
+                  outputColumnNames: _col0, _col1, _col2
+                  Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: char(16))
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: char(16))
+                    Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col1 (type: bigint), _col2 (type: double)
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Left Semi Join 0 to 1
+                keys:
+                  0 _col44 (type: date)
+                  1 _col0 (type: date)
+                outputColumnNames: _col10, _col21
+                Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: sum(_col10)
+                  keys: _col21 (type: char(16))
+                  minReductionHashAggr: 0.99
+                  mode: hash
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: char(16))
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: char(16))
+                    Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col1 (type: bigint)
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: sum(VALUE._col0)
+                keys: KEY._col0 (type: char(16))
+                mode: mergepartial
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: NONE
+                Select Operator
+                  expressions: _col0 (type: char(16)), _col1 (type: bigint), UDFToDouble(_col1) (type: double)
+                  outputColumnNames: _col0, _col1, _col2
+                  Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: char(16))
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: char(16))
+                    Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col1 (type: bigint), _col2 (type: double)
+        Reducer 4 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: char(16))
+                  1 _col0 (type: char(16))
+                outputColumnNames: _col0, _col1, _col2, _col4, _col5
+                Statistics: Num rows: 1 Data size: 193 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col0 (type: char(16))
+                  sort order: +
+                  Map-reduce partition columns: _col0 (type: char(16))
+                  Statistics: Num rows: 1 Data size: 193 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col1 (type: bigint), _col2 (type: double), _col4 (type: bigint), _col5 (type: double)
+        Reducer 5 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: char(16))
+                  1 _col0 (type: char(16))
+                outputColumnNames: _col0, _col1, _col2, _col4, _col5, _col7, _col8
+                Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: NONE
+                Select Operator
+                  expressions: _col0 (type: char(16)), _col1 (type: bigint), (((_col2 / UDFToDouble(((_col1 + _col4) + _col7))) / 3.0D) * 100.0D) (type: double), _col4 (type: bigint), (((_col5 / UDFToDouble(((_col1 + _col4) + _col7))) / 3.0D) * 100.0D) (type: double), _col7 (type: bigint), (((_col8 / UDFToDouble(((_col1 + _col4) + _col7))) / 3.0D) * 100.0D) (type: double), (CAST( ((_col1 + _col4) + _col7) AS decimal(19,0)) / 3) (type: decimal(25,6))
+                  outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
+                  Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: char(16)), _col1 (type: bigint)
+                    sort order: ++
+                    Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: NONE
+                    TopN Hash Memory Usage: 0.1
+                    value expressions: _col2 (type: double), _col3 (type: bigint), _col4 (type: double), _col5 (type: bigint), _col6 (type: double), _col7 (type: decimal(25,6))
+        Reducer 6 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: KEY.reducesinkkey0 (type: char(16)), KEY.reducesinkkey1 (type: bigint), VALUE._col0 (type: double), VALUE._col1 (type: bigint), VALUE._col2 (type: double), VALUE._col3 (type: bigint), VALUE._col4 (type: double), VALUE._col5 (type: decimal(25,6))
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
+                Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: NONE
+                Limit
+                  Number of rows: 100
+                  Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: NONE
+                  File Output Operator
+                    compressed: false
+                    Statistics: Num rows: 1 Data size: 212 Basic stats: COMPLETE Column stats: NONE
+                    table:
+                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 8 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Left Semi Join 0 to 1
+                keys:
+                  0 _col51 (type: date)
+                  1 _col0 (type: date)
+                outputColumnNames: _col17, _col28
+                Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: sum(_col17)
+                  keys: _col28 (type: char(16))
+                  minReductionHashAggr: 0.99
+                  mode: hash
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: char(16))
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: char(16))
+                    Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col1 (type: bigint)
+        Reducer 9 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: sum(VALUE._col0)
+                keys: KEY._col0 (type: char(16))
+                mode: mergepartial
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: NONE
+                Select Operator
+                  expressions: _col0 (type: char(16)), _col1 (type: bigint), UDFToDouble(_col1) (type: double)
+                  outputColumnNames: _col0, _col1, _col2
+                  Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: char(16))
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: char(16))
+                    Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col1 (type: bigint), _col2 (type: double)
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: 100
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: explain
+with ss as (
+			 select
+				  i_manufact_id,sum(ss_ext_sales_price) total_sales
+			 from
+				store_sales,
+				date_dim,
+				 customer_address,
+				 item
+			 where
+				 i_manufact_id in (select
+			  i_manufact_id
+			from
+			 item
+			where i_category in ('Books'))
+			 and     ss_item_sk              = i_item_sk
+			 and     ss_sold_date_sk         = d_date_sk
+			 and     d_year                  = 1999
+			 and     d_moy                   = 3
+			 and     ss_addr_sk              = ca_address_sk
+			 and     ca_gmt_offset           = -6 
+			 group by i_manufact_id),
+			 cs as (
+			 select
+				  i_manufact_id,sum(cs_ext_sales_price) total_sales
+			 from
+				catalog_sales,
+				date_dim,
+				 customer_address,
+				 item
+			 where
+				 i_manufact_id               in (select
+			  i_manufact_id
+			from
+			 item
+			where i_category in ('Books'))
+			 and     cs_item_sk              = i_item_sk
+			 and     cs_sold_date_sk         = d_date_sk
+			 and     d_year                  = 1999
+			 and     d_moy                   = 3
+			 and     cs_bill_addr_sk         = ca_address_sk
+			 and     ca_gmt_offset           = -6 
+			 group by i_manufact_id),
+			 ws as (
+			 select
+				  i_manufact_id,sum(ws_ext_sales_price) total_sales
+			 from
+				web_sales,
+				date_dim,
+				 customer_address,
+				 item
+			 where
+				 i_manufact_id               in (select
+			  i_manufact_id
+			from
+			 item
+			where i_category in ('Books'))
+			 and     ws_item_sk              = i_item_sk
+			 and     ws_sold_date_sk         = d_date_sk
+			 and     d_year                  = 1999
+			 and     d_moy                   = 3
+			 and     ws_bill_addr_sk         = ca_address_sk
+			 and     ca_gmt_offset           = -6
+			 group by i_manufact_id)
+			  select  i_manufact_id ,sum(total_sales) total_sales
+			 from  (select * from ss 
+				union all
+				select * from cs 
+				union all
+				select * from ws) tmp1
+			 group by i_manufact_id
+			 order by total_sales
+			limit 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@catalog_sales
+PREHOOK: Input: default@customer_address
+PREHOOK: Input: default@date_dim
+PREHOOK: Input: default@item
+PREHOOK: Input: default@store_sales
+PREHOOK: Input: default@web_sales
+#### A masked pattern was here ####
+POSTHOOK: query: explain
+with ss as (
+			 select
+				  i_manufact_id,sum(ss_ext_sales_price) total_sales
+			 from
+				store_sales,
+				date_dim,
+				 customer_address,
+				 item
+			 where
+				 i_manufact_id in (select
+			  i_manufact_id
+			from
+			 item
+			where i_category in ('Books'))
+			 and     ss_item_sk              = i_item_sk
+			 and     ss_sold_date_sk         = d_date_sk
+			 and     d_year                  = 1999
+			 and     d_moy                   = 3
+			 and     ss_addr_sk              = ca_address_sk
+			 and     ca_gmt_offset           = -6 
+			 group by i_manufact_id),
+			 cs as (
+			 select
+				  i_manufact_id,sum(cs_ext_sales_price) total_sales
+			 from
+				catalog_sales,
+				date_dim,
+				 customer_address,
+				 item
+			 where
+				 i_manufact_id               in (select
+			  i_manufact_id
+			from
+			 item
+			where i_category in ('Books'))
+			 and     cs_item_sk              = i_item_sk
+			 and     cs_sold_date_sk         = d_date_sk
+			 and     d_year                  = 1999
+			 and     d_moy                   = 3
+			 and     cs_bill_addr_sk         = ca_address_sk
+			 and     ca_gmt_offset           = -6 
+			 group by i_manufact_id),
+			 ws as (
+			 select
+				  i_manufact_id,sum(ws_ext_sales_price) total_sales
+			 from
+				web_sales,
+				date_dim,
+				 customer_address,
+				 item
+			 where
+				 i_manufact_id               in (select
+			  i_manufact_id
+			from
+			 item
+			where i_category in ('Books'))
+			 and     ws_item_sk              = i_item_sk
+			 and     ws_sold_date_sk         = d_date_sk
+			 and     d_year                  = 1999
+			 and     d_moy                   = 3
+			 and     ws_bill_addr_sk         = ca_address_sk
+			 and     ca_gmt_offset           = -6
+			 group by i_manufact_id)
+			  select  i_manufact_id ,sum(total_sales) total_sales
+			 from  (select * from ss 
+				union all
+				select * from cs 
+				union all
+				select * from ws) tmp1
+			 group by i_manufact_id
+			 order by total_sales
+			limit 100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@catalog_sales
+POSTHOOK: Input: default@customer_address
+POSTHOOK: Input: default@date_dim
+POSTHOOK: Input: default@item
+POSTHOOK: Input: default@store_sales
+POSTHOOK: Input: default@web_sales
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 10 <- Map 13 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
+        Reducer 11 <- Reducer 10 (SIMPLE_EDGE), Union 4 (CONTAINS)
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Union 4 (CONTAINS)
+        Reducer 5 <- Union 4 (SIMPLE_EDGE)
+        Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
+        Reducer 8 <- Map 12 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
+        Reducer 9 <- Reducer 8 (SIMPLE_EDGE), Union 4 (CONTAINS)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: store_sales
+                  properties:
+                    hive.sql.query SELECT "t0"."ss_sold_date_sk", "t0"."ss_item_sk", "t0"."ss_addr_sk", "t0"."ss_ext_sales_price", "t2"."d_date_sk", "t4"."ca_address_sk", "t6"."i_item_sk", "t6"."i_item_id", "t6"."i_rec_start_date", "t6"."i_rec_end_date", "t6"."i_item_desc", "t6"."i_current_price", "t6"."i_wholesale_cost", "t6"."i_brand_id", "t6"."i_brand", "t6"."i_class_id", "t6"."i_class", "t6"."i_category_id", "t6"."i_category", "t6"."i_manufact_id", "t6"."i_manufact", "t6"."i_size", " [...]
+FROM (SELECT "ss_sold_date_sk", "ss_item_sk", "ss_addr_sk", "ss_ext_sales_price"
+FROM "STORE_SALES"
+WHERE "ss_sold_date_sk" IS NOT NULL AND "ss_addr_sk" IS NOT NULL AND "ss_item_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "d_date_sk"
+FROM "DATE_DIM"
+WHERE "d_year" = 1999 AND "d_moy" = 3 AND "d_date_sk" IS NOT NULL) AS "t2" ON "t0"."ss_sold_date_sk" = "t2"."d_date_sk"
+INNER JOIN (SELECT "ca_address_sk"
+FROM "CUSTOMER_ADDRESS"
+WHERE "ca_gmt_offset" = -6 AND "ca_address_sk" IS NOT NULL) AS "t4" ON "t0"."ss_addr_sk" = "t4"."ca_address_sk"
+INNER JOIN (SELECT "i_item_sk", "i_item_id", "i_rec_start_date", "i_rec_end_date", "i_item_desc", "i_current_price", "i_wholesale_cost", "i_brand_id", "i_brand", "i_class_id", "i_class", "i_category_id", "i_category", "i_manufact_id", "i_manufact", "i_size", "i_formulation", "i_color", "i_units", "i_container", "i_manager_id", "i_product_name"
+FROM "ITEM"
+WHERE "i_manufact_id" IS NOT NULL AND "i_item_sk" IS NOT NULL) AS "t6" ON "t0"."ss_item_sk" = "t6"."i_item_sk"
+                    hive.sql.query.fieldNames ss_sold_date_sk,ss_item_sk,ss_addr_sk,ss_ext_sales_price,d_date_sk,ca_address_sk,i_item_sk,i_item_id,i_rec_start_date,i_rec_end_date,i_item_desc,i_current_price,i_wholesale_cost,i_brand_id,i_brand,i_class_id,i_class,i_category_id,i_category,i_manufact_id,i_manufact,i_size,i_formulation,i_color,i_units,i_container,i_manager_id,i_product_name
+                    hive.sql.query.fieldTypes int,int,int,decimal(7,2),int,int,int,char(16),date,date,varchar(200),decimal(7,2),decimal(7,2),int,char(50),int,char(50),int,char(50),int,char(50),char(20),char(20),char(20),char(10),char(10),int,char(50)
+                    hive.sql.query.split false
+                  Statistics: Num rows: 1 Data size: 116 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: ss_ext_sales_price (type: decimal(7,2)), i_manufact_id (type: int)
+                    outputColumnNames: _col3, _col19
+                    Statistics: Num rows: 1 Data size: 116 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: _col19 (type: int)
+                      sort order: +
+                      Map-reduce partition columns: _col19 (type: int)
+                      Statistics: Num rows: 1 Data size: 116 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col3 (type: decimal(7,2))
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Map 12 
+            Map Operator Tree:
+                TableScan
+                  alias: catalog_sales
+                  properties:
+                    hive.sql.query SELECT "t0"."cs_sold_date_sk", "t0"."cs_bill_addr_sk", "t0"."cs_item_sk", "t0"."cs_ext_sales_price", "t2"."d_date_sk", "t4"."ca_address_sk", "t6"."i_item_sk", "t6"."i_item_id", "t6"."i_rec_start_date", "t6"."i_rec_end_date", "t6"."i_item_desc", "t6"."i_current_price", "t6"."i_wholesale_cost", "t6"."i_brand_id", "t6"."i_brand", "t6"."i_class_id", "t6"."i_class", "t6"."i_category_id", "t6"."i_category", "t6"."i_manufact_id", "t6"."i_manufact", "t6"."i_siz [...]
+FROM (SELECT "cs_sold_date_sk", "cs_bill_addr_sk", "cs_item_sk", "cs_ext_sales_price"
+FROM "CATALOG_SALES"
+WHERE "cs_sold_date_sk" IS NOT NULL AND "cs_bill_addr_sk" IS NOT NULL AND "cs_item_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "d_date_sk"
+FROM "DATE_DIM"
+WHERE "d_year" = 1999 AND "d_moy" = 3 AND "d_date_sk" IS NOT NULL) AS "t2" ON "t0"."cs_sold_date_sk" = "t2"."d_date_sk"
+INNER JOIN (SELECT "ca_address_sk"
+FROM "CUSTOMER_ADDRESS"
+WHERE "ca_gmt_offset" = -6 AND "ca_address_sk" IS NOT NULL) AS "t4" ON "t0"."cs_bill_addr_sk" = "t4"."ca_address_sk"
+INNER JOIN (SELECT "i_item_sk", "i_item_id", "i_rec_start_date", "i_rec_end_date", "i_item_desc", "i_current_price", "i_wholesale_cost", "i_brand_id", "i_brand", "i_class_id", "i_class", "i_category_id", "i_category", "i_manufact_id", "i_manufact", "i_size", "i_formulation", "i_color", "i_units", "i_container", "i_manager_id", "i_product_name"
+FROM "ITEM"
+WHERE "i_manufact_id" IS NOT NULL AND "i_item_sk" IS NOT NULL) AS "t6" ON "t0"."cs_item_sk" = "t6"."i_item_sk"
+                    hive.sql.query.fieldNames cs_sold_date_sk,cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,d_date_sk,ca_address_sk,i_item_sk,i_item_id,i_rec_start_date,i_rec_end_date,i_item_desc,i_current_price,i_wholesale_cost,i_brand_id,i_brand,i_class_id,i_class,i_category_id,i_category,i_manufact_id,i_manufact,i_size,i_formulation,i_color,i_units,i_container,i_manager_id,i_product_name
+                    hive.sql.query.fieldTypes int,int,int,decimal(7,2),int,int,int,char(16),date,date,varchar(200),decimal(7,2),decimal(7,2),int,char(50),int,char(50),int,char(50),int,char(50),char(20),char(20),char(20),char(10),char(10),int,char(50)
+                    hive.sql.query.split false
+                  Statistics: Num rows: 1 Data size: 116 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: cs_ext_sales_price (type: decimal(7,2)), i_manufact_id (type: int)
+                    outputColumnNames: _col3, _col19
+                    Statistics: Num rows: 1 Data size: 116 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: _col19 (type: int)
+                      sort order: +
+                      Map-reduce partition columns: _col19 (type: int)
+                      Statistics: Num rows: 1 Data size: 116 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col3 (type: decimal(7,2))
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Map 13 
+            Map Operator Tree:
+                TableScan
+                  alias: web_sales
+                  properties:
+                    hive.sql.query SELECT "t0"."ws_sold_date_sk", "t0"."ws_item_sk", "t0"."ws_bill_addr_sk", "t0"."ws_ext_sales_price", "t2"."d_date_sk", "t4"."ca_address_sk", "t6"."i_item_sk", "t6"."i_item_id", "t6"."i_rec_start_date", "t6"."i_rec_end_date", "t6"."i_item_desc", "t6"."i_current_price", "t6"."i_wholesale_cost", "t6"."i_brand_id", "t6"."i_brand", "t6"."i_class_id", "t6"."i_class", "t6"."i_category_id", "t6"."i_category", "t6"."i_manufact_id", "t6"."i_manufact", "t6"."i_siz [...]
+FROM (SELECT "ws_sold_date_sk", "ws_item_sk", "ws_bill_addr_sk", "ws_ext_sales_price"
+FROM "WEB_SALES"
+WHERE "ws_sold_date_sk" IS NOT NULL AND "ws_bill_addr_sk" IS NOT NULL AND "ws_item_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "d_date_sk"
+FROM "DATE_DIM"
+WHERE "d_year" = 1999 AND "d_moy" = 3 AND "d_date_sk" IS NOT NULL) AS "t2" ON "t0"."ws_sold_date_sk" = "t2"."d_date_sk"
+INNER JOIN (SELECT "ca_address_sk"
+FROM "CUSTOMER_ADDRESS"
+WHERE "ca_gmt_offset" = -6 AND "ca_address_sk" IS NOT NULL) AS "t4" ON "t0"."ws_bill_addr_sk" = "t4"."ca_address_sk"
+INNER JOIN (SELECT "i_item_sk", "i_item_id", "i_rec_start_date", "i_rec_end_date", "i_item_desc", "i_current_price", "i_wholesale_cost", "i_brand_id", "i_brand", "i_class_id", "i_class", "i_category_id", "i_category", "i_manufact_id", "i_manufact", "i_size", "i_formulation", "i_color", "i_units", "i_container", "i_manager_id", "i_product_name"
+FROM "ITEM"
+WHERE "i_manufact_id" IS NOT NULL AND "i_item_sk" IS NOT NULL) AS "t6" ON "t0"."ws_item_sk" = "t6"."i_item_sk"
+                    hive.sql.query.fieldNames ws_sold_date_sk,ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,d_date_sk,ca_address_sk,i_item_sk,i_item_id,i_rec_start_date,i_rec_end_date,i_item_desc,i_current_price,i_wholesale_cost,i_brand_id,i_brand,i_class_id,i_class,i_category_id,i_category,i_manufact_id,i_manufact,i_size,i_formulation,i_color,i_units,i_container,i_manager_id,i_product_name
+                    hive.sql.query.fieldTypes int,int,int,decimal(7,2),int,int,int,char(16),date,date,varchar(200),decimal(7,2),decimal(7,2),int,char(50),int,char(50),int,char(50),int,char(50),char(20),char(20),char(20),char(10),char(10),int,char(50)
+                    hive.sql.query.split false
+                  Statistics: Num rows: 1 Data size: 116 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: ws_ext_sales_price (type: decimal(7,2)), i_manufact_id (type: int)
+                    outputColumnNames: _col3, _col19
+                    Statistics: Num rows: 1 Data size: 116 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: _col19 (type: int)
+                      sort order: +
+                      Map-reduce partition columns: _col19 (type: int)
+                      Statistics: Num rows: 1 Data size: 116 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col3 (type: decimal(7,2))
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Map 7 
+            Map Operator Tree:
+                TableScan
+                  alias: item
+                  properties:
+                    hive.sql.query SELECT "i_manufact_id"
+FROM "ITEM"
+WHERE "i_category" = 'Books                                             ' AND "i_manufact_id" IS NOT NULL
+                    hive.sql.query.fieldNames i_manufact_id
+                    hive.sql.query.fieldTypes int
+                    hive.sql.query.split true
+                  Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: i_manufact_id (type: int)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                    Group By Operator
+                      keys: _col0 (type: int)
+                      minReductionHashAggr: 0.99
+                      mode: hash
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Reducer 10 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Left Semi Join 0 to 1
+                keys:
+                  0 _col19 (type: int)
+                  1 _col0 (type: int)
+                outputColumnNames: _col3, _col19
+                Statistics: Num rows: 1 Data size: 127 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: sum(_col3)
+                  keys: _col19 (type: int)
+                  minReductionHashAggr: 0.99
+                  mode: hash
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 1 Data size: 127 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: int)
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: int)
+                    Statistics: Num rows: 1 Data size: 127 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col1 (type: decimal(17,2))
+        Reducer 11 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: sum(VALUE._col0)
+                keys: KEY._col0 (type: int)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 127 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: sum(_col1)
+                  keys: _col0 (type: int)
+                  minReductionHashAggr: 0.99
+                  mode: hash
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 3 Data size: 381 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: int)
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: int)
+                    Statistics: Num rows: 3 Data size: 381 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col1 (type: decimal(27,2))
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Left Semi Join 0 to 1
+                keys:
+                  0 _col19 (type: int)
+                  1 _col0 (type: int)
+                outputColumnNames: _col3, _col19
+                Statistics: Num rows: 1 Data size: 127 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: sum(_col3)
+                  keys: _col19 (type: int)
+                  minReductionHashAggr: 0.99
+                  mode: hash
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 1 Data size: 127 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: int)
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: int)
+                    Statistics: Num rows: 1 Data size: 127 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col1 (type: decimal(17,2))
+        Reducer 3 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: sum(VALUE._col0)
+                keys: KEY._col0 (type: int)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 127 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: sum(_col1)
+                  keys: _col0 (type: int)
+                  minReductionHashAggr: 0.99
+                  mode: hash
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 3 Data size: 381 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: int)
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: int)
+                    Statistics: Num rows: 3 Data size: 381 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col1 (type: decimal(27,2))
+        Reducer 5 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: sum(VALUE._col0)
+                keys: KEY._col0 (type: int)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 127 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col1 (type: decimal(27,2))
+                  sort order: +
+                  Statistics: Num rows: 1 Data size: 127 Basic stats: COMPLETE Column stats: NONE
+                  TopN Hash Memory Usage: 0.1
+                  value expressions: _col0 (type: int)
+        Reducer 6 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: VALUE._col0 (type: int), KEY.reducesinkkey0 (type: decimal(27,2))
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 127 Basic stats: COMPLETE Column stats: NONE
+                Limit
+                  Number of rows: 100
+                  Statistics: Num rows: 1 Data size: 127 Basic stats: COMPLETE Column stats: NONE
+                  File Output Operator
+                    compressed: false
+                    Statistics: Num rows: 1 Data size: 127 Basic stats: COMPLETE Column stats: NONE
+                    table:
+                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+        Reducer 8 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Left Semi Join 0 to 1
+                keys:
+                  0 _col19 (type: int)
+                  1 _col0 (type: int)
+                outputColumnNames: _col3, _col19
+                Statistics: Num rows: 1 Data size: 127 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: sum(_col3)
+                  keys: _col19 (type: int)
+                  minReductionHashAggr: 0.99
+                  mode: hash
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 1 Data size: 127 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: int)
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: int)
+                    Statistics: Num rows: 1 Data size: 127 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col1 (type: decimal(17,2))
+        Reducer 9 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: sum(VALUE._col0)
+                keys: KEY._col0 (type: int)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 127 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: sum(_col1)
+                  keys: _col0 (type: int)
+                  minReductionHashAggr: 0.99
+                  mode: hash
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 3 Data size: 381 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: int)
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: int)
+                    Statistics: Num rows: 3 Data size: 381 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col1 (type: decimal(27,2))
+        Union 4 
+            Vertex: Union 4
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: 100
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: with ss as (
+			 select
+				  i_manufact_id,sum(ss_ext_sales_price) total_sales
+			 from
+				store_sales,
+				date_dim,
+				 customer_address,
+				 item
+			 where
+				 i_manufact_id in (select
+			  i_manufact_id
+			from
+			 item
+			where i_category in ('Books'))
+			 and     ss_item_sk              = i_item_sk
+			 and     ss_sold_date_sk         = d_date_sk
+			 and     d_year                  = 1999
+			 and     d_moy                   = 3
+			 and     ss_addr_sk              = ca_address_sk
+			 and     ca_gmt_offset           = -6 
+			 group by i_manufact_id),
+			 cs as (
+			 select
+				  i_manufact_id,sum(cs_ext_sales_price) total_sales
+			 from
+				catalog_sales,
+				date_dim,
+				 customer_address,
+				 item
+			 where
+				 i_manufact_id               in (select
+			  i_manufact_id
+			from
+			 item
+			where i_category in ('Books'))
+			 and     cs_item_sk              = i_item_sk
+			 and     cs_sold_date_sk         = d_date_sk
+			 and     d_year                  = 1999
+			 and     d_moy                   = 3
+			 and     cs_bill_addr_sk         = ca_address_sk
+			 and     ca_gmt_offset           = -6 
+			 group by i_manufact_id),
+			 ws as (
+			 select
+				  i_manufact_id,sum(ws_ext_sales_price) total_sales
+			 from
+				web_sales,
+				date_dim,
+				 customer_address,
+				 item
+			 where
+				 i_manufact_id               in (select
+			  i_manufact_id
+			from
+			 item
+			where i_category in ('Books'))
+			 and     ws_item_sk              = i_item_sk
+			 and     ws_sold_date_sk         = d_date_sk
+			 and     d_year                  = 1999
+			 and     d_moy                   = 3
+			 and     ws_bill_addr_sk         = ca_address_sk
+			 and     ca_gmt_offset           = -6
+			 group by i_manufact_id)
+			  select  i_manufact_id ,sum(total_sales) total_sales
+			 from  (select * from ss 
+				union all
+				select * from cs 
+				union all
+				select * from ws) tmp1
+			 group by i_manufact_id
+			 order by total_sales
+			limit 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@catalog_sales
+PREHOOK: Input: default@customer_address
+PREHOOK: Input: default@date_dim
+PREHOOK: Input: default@item
+PREHOOK: Input: default@store_sales
+PREHOOK: Input: default@web_sales
+#### A masked pattern was here ####
+POSTHOOK: query: with ss as (
+			 select
+				  i_manufact_id,sum(ss_ext_sales_price) total_sales
+			 from
+				store_sales,
+				date_dim,
+				 customer_address,
+				 item
+			 where
+				 i_manufact_id in (select
+			  i_manufact_id
+			from
+			 item
+			where i_category in ('Books'))
+			 and     ss_item_sk              = i_item_sk
+			 and     ss_sold_date_sk         = d_date_sk
+			 and     d_year                  = 1999
+			 and     d_moy                   = 3
+			 and     ss_addr_sk              = ca_address_sk
+			 and     ca_gmt_offset           = -6 
+			 group by i_manufact_id),
+			 cs as (
+			 select
+				  i_manufact_id,sum(cs_ext_sales_price) total_sales
+			 from
+				catalog_sales,
+				date_dim,
+				 customer_address,
+				 item
+			 where
+				 i_manufact_id               in (select
+			  i_manufact_id
+			from
+			 item
+			where i_category in ('Books'))
+			 and     cs_item_sk              = i_item_sk
+			 and     cs_sold_date_sk         = d_date_sk
+			 and     d_year                  = 1999
+			 and     d_moy                   = 3
+			 and     cs_bill_addr_sk         = ca_address_sk
+			 and     ca_gmt_offset           = -6 
+			 group by i_manufact_id),
+			 ws as (
+			 select
+				  i_manufact_id,sum(ws_ext_sales_price) total_sales
+			 from
+				web_sales,
+				date_dim,
+				 customer_address,
+				 item
+			 where
+				 i_manufact_id               in (select
+			  i_manufact_id
+			from
+			 item
+			where i_category in ('Books'))
+			 and     ws_item_sk              = i_item_sk
+			 and     ws_sold_date_sk         = d_date_sk
+			 and     d_year                  = 1999
+			 and     d_moy                   = 3
+			 and     ws_bill_addr_sk         = ca_address_sk
+			 and     ca_gmt_offset           = -6
+			 group by i_manufact_id)
+			  select  i_manufact_id ,sum(total_sales) total_sales
+			 from  (select * from ss 
+				union all
+				select * from cs 
+				union all
+				select * from ws) tmp1
+			 group by i_manufact_id
+			 order by total_sales
+			limit 100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@catalog_sales
+POSTHOOK: Input: default@customer_address
+POSTHOOK: Input: default@date_dim
+POSTHOOK: Input: default@item
+POSTHOOK: Input: default@store_sales
+POSTHOOK: Input: default@web_sales
+#### A masked pattern was here ####
+Warning: Shuffle Join MERGEJOIN[40][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+Warning: Shuffle Join MERGEJOIN[41][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[42][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[43][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 5' is a cross product
+Warning: Shuffle Join MERGEJOIN[44][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5]] in Stage 'Reducer 6' is a cross product
+Warning: Shuffle Join MERGEJOIN[45][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5, $hdt$_6]] in Stage 'Reducer 7' is a cross product
+Warning: Shuffle Join MERGEJOIN[46][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5, $hdt$_6, $hdt$_7]] in Stage 'Reducer 8' is a cross product
+PREHOOK: query: explain
+select *
+                          from (select count(*) as h8_30_to_9
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 8
+                                  and time_dim.t_minute >= 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s1 join
+                               (select count(*) as h9_to_9_30
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 9
+                                  and time_dim.t_minute < 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s2 join
+                               (select count(*) as h9_30_to_10
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 9
+                                  and time_dim.t_minute >= 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s3 join
+                               (select count(*) as h10_to_10_30
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 10
+                                  and time_dim.t_minute < 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s4 join
+                               (select count(*) as h10_30_to_11
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 10
+                                  and time_dim.t_minute >= 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s5 join
+                               (select count(*) as h11_to_11_30
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 11
+                                  and time_dim.t_minute < 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s6 join
+                               (select count(*) as h11_30_to_12
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 11
+                                  and time_dim.t_minute >= 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s7 join
+                               (select count(*) as h12_to_12_30
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 12
+                                  and time_dim.t_minute < 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s8
+PREHOOK: type: QUERY
+PREHOOK: Input: default@household_demographics
+PREHOOK: Input: default@store
+PREHOOK: Input: default@store_sales
+PREHOOK: Input: default@time_dim
+#### A masked pattern was here ####
+POSTHOOK: query: explain
+select *
+                          from (select count(*) as h8_30_to_9
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 8
+                                  and time_dim.t_minute >= 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s1 join
+                               (select count(*) as h9_to_9_30
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 9
+                                  and time_dim.t_minute < 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s2 join
+                               (select count(*) as h9_30_to_10
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 9
+                                  and time_dim.t_minute >= 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s3 join
+                               (select count(*) as h10_to_10_30
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 10
+                                  and time_dim.t_minute < 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s4 join
+                               (select count(*) as h10_30_to_11
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 10
+                                  and time_dim.t_minute >= 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s5 join
+                               (select count(*) as h11_to_11_30
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 11
+                                  and time_dim.t_minute < 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s6 join
+                               (select count(*) as h11_30_to_12
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 11
+                                  and time_dim.t_minute >= 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s7 join
+                               (select count(*) as h12_to_12_30
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 12
+                                  and time_dim.t_minute < 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s8
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@household_demographics
+POSTHOOK: Input: default@store
+POSTHOOK: Input: default@store_sales
+POSTHOOK: Input: default@time_dim
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (XPROD_EDGE), Map 9 (XPROD_EDGE)
+        Reducer 3 <- Map 10 (XPROD_EDGE), Reducer 2 (XPROD_EDGE)
+        Reducer 4 <- Map 11 (XPROD_EDGE), Reducer 3 (XPROD_EDGE)
+        Reducer 5 <- Map 12 (XPROD_EDGE), Reducer 4 (XPROD_EDGE)
+        Reducer 6 <- Map 13 (XPROD_EDGE), Reducer 5 (XPROD_EDGE)
+        Reducer 7 <- Map 14 (XPROD_EDGE), Reducer 6 (XPROD_EDGE)
+        Reducer 8 <- Map 15 (XPROD_EDGE), Reducer 7 (XPROD_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: store_sales
+                  properties:
+                    hive.sql.query SELECT COUNT(*) AS "$f0"
+FROM (SELECT "ss_sold_time_sk", "ss_hdemo_sk", "ss_store_sk"
+FROM "STORE_SALES"
+WHERE "ss_hdemo_sk" IS NOT NULL AND "ss_sold_time_sk" IS NOT NULL AND "ss_store_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "hd_demo_sk"
+FROM "HOUSEHOLD_DEMOGRAPHICS"
+WHERE "hd_dep_count" IN (4, 2, 0) AND ("hd_vehicle_count" <= 5 OR "hd_vehicle_count" <= 2 OR "hd_vehicle_count" <= 3) AND ("hd_dep_count" = 4 AND "hd_vehicle_count" <= 5 OR "hd_dep_count" = 2 AND "hd_vehicle_count" <= 2 OR "hd_dep_count" = 0 AND "hd_vehicle_count" <= 3) AND "hd_demo_sk" IS NOT NULL) AS "t2" ON "t0"."ss_hdemo_sk" = "t2"."hd_demo_sk"
+INNER JOIN (SELECT "t_time_sk"
+FROM "TIME_DIM"
+WHERE "t_hour" = 8 AND "t_minute" >= 30 AND "t_time_sk" IS NOT NULL) AS "t4" ON "t0"."ss_sold_time_sk" = "t4"."t_time_sk"
+INNER JOIN (SELECT "s_store_sk"
+FROM "STORE"
+WHERE "s_store_name" = 'ese' AND "s_store_sk" IS NOT NULL) AS "t6" ON "t0"."ss_store_sk" = "t6"."s_store_sk"
+                    hive.sql.query.fieldNames $f0
+                    hive.sql.query.fieldTypes bigint
+                    hive.sql.query.split false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: $f0 (type: bigint)
+                    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)
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Map 10 
+            Map Operator Tree:
+                TableScan
+                  alias: store_sales
+                  properties:
+                    hive.sql.query SELECT COUNT(*) AS "$f0"
+FROM (SELECT "ss_sold_time_sk", "ss_hdemo_sk", "ss_store_sk"
+FROM "STORE_SALES"
+WHERE "ss_hdemo_sk" IS NOT NULL AND "ss_sold_time_sk" IS NOT NULL AND "ss_store_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "hd_demo_sk"
+FROM "HOUSEHOLD_DEMOGRAPHICS"
+WHERE "hd_dep_count" IN (4, 2, 0) AND ("hd_vehicle_count" <= 5 OR "hd_vehicle_count" <= 2 OR "hd_vehicle_count" <= 3) AND ("hd_dep_count" = 4 AND "hd_vehicle_count" <= 5 OR "hd_dep_count" = 2 AND "hd_vehicle_count" <= 2 OR "hd_dep_count" = 0 AND "hd_vehicle_count" <= 3) AND "hd_demo_sk" IS NOT NULL) AS "t2" ON "t0"."ss_hdemo_sk" = "t2"."hd_demo_sk"
+INNER JOIN (SELECT "t_time_sk"
+FROM "TIME_DIM"
+WHERE "t_hour" = 9 AND "t_minute" >= 30 AND "t_time_sk" IS NOT NULL) AS "t4" ON "t0"."ss_sold_time_sk" = "t4"."t_time_sk"
+INNER JOIN (SELECT "s_store_sk"
+FROM "STORE"
+WHERE "s_store_name" = 'ese' AND "s_store_sk" IS NOT NULL) AS "t6" ON "t0"."ss_store_sk" = "t6"."s_store_sk"
+                    hive.sql.query.fieldNames $f0
+                    hive.sql.query.fieldTypes bigint
+                    hive.sql.query.split false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: $f0 (type: bigint)
+                    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)
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Map 11 
+            Map Operator Tree:
+                TableScan
+                  alias: store_sales
+                  properties:
+                    hive.sql.query SELECT COUNT(*) AS "$f0"
+FROM (SELECT "ss_sold_time_sk", "ss_hdemo_sk", "ss_store_sk"
+FROM "STORE_SALES"
+WHERE "ss_hdemo_sk" IS NOT NULL AND "ss_sold_time_sk" IS NOT NULL AND "ss_store_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "hd_demo_sk"
+FROM "HOUSEHOLD_DEMOGRAPHICS"
+WHERE "hd_dep_count" IN (4, 2, 0) AND ("hd_vehicle_count" <= 5 OR "hd_vehicle_count" <= 2 OR "hd_vehicle_count" <= 3) AND ("hd_dep_count" = 4 AND "hd_vehicle_count" <= 5 OR "hd_dep_count" = 2 AND "hd_vehicle_count" <= 2 OR "hd_dep_count" = 0 AND "hd_vehicle_count" <= 3) AND "hd_demo_sk" IS NOT NULL) AS "t2" ON "t0"."ss_hdemo_sk" = "t2"."hd_demo_sk"
+INNER JOIN (SELECT "t_time_sk"
+FROM "TIME_DIM"
+WHERE "t_hour" = 10 AND "t_minute" < 30 AND "t_time_sk" IS NOT NULL) AS "t4" ON "t0"."ss_sold_time_sk" = "t4"."t_time_sk"
+INNER JOIN (SELECT "s_store_sk"
+FROM "STORE"
+WHERE "s_store_name" = 'ese' AND "s_store_sk" IS NOT NULL) AS "t6" ON "t0"."ss_store_sk" = "t6"."s_store_sk"
+                    hive.sql.query.fieldNames $f0
+                    hive.sql.query.fieldTypes bigint
+                    hive.sql.query.split false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: $f0 (type: bigint)
+                    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)
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Map 12 
+            Map Operator Tree:
+                TableScan
+                  alias: store_sales
+                  properties:
+                    hive.sql.query SELECT COUNT(*) AS "$f0"
+FROM (SELECT "ss_sold_time_sk", "ss_hdemo_sk", "ss_store_sk"
+FROM "STORE_SALES"
+WHERE "ss_hdemo_sk" IS NOT NULL AND "ss_sold_time_sk" IS NOT NULL AND "ss_store_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "hd_demo_sk"
+FROM "HOUSEHOLD_DEMOGRAPHICS"
+WHERE "hd_dep_count" IN (4, 2, 0) AND ("hd_vehicle_count" <= 5 OR "hd_vehicle_count" <= 2 OR "hd_vehicle_count" <= 3) AND ("hd_dep_count" = 4 AND "hd_vehicle_count" <= 5 OR "hd_dep_count" = 2 AND "hd_vehicle_count" <= 2 OR "hd_dep_count" = 0 AND "hd_vehicle_count" <= 3) AND "hd_demo_sk" IS NOT NULL) AS "t2" ON "t0"."ss_hdemo_sk" = "t2"."hd_demo_sk"
+INNER JOIN (SELECT "t_time_sk"
+FROM "TIME_DIM"
+WHERE "t_hour" = 10 AND "t_minute" >= 30 AND "t_time_sk" IS NOT NULL) AS "t4" ON "t0"."ss_sold_time_sk" = "t4"."t_time_sk"
+INNER JOIN (SELECT "s_store_sk"
+FROM "STORE"
+WHERE "s_store_name" = 'ese' AND "s_store_sk" IS NOT NULL) AS "t6" ON "t0"."ss_store_sk" = "t6"."s_store_sk"
+                    hive.sql.query.fieldNames $f0
+                    hive.sql.query.fieldTypes bigint
+                    hive.sql.query.split false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: $f0 (type: bigint)
+                    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)
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Map 13 
+            Map Operator Tree:
+                TableScan
+                  alias: store_sales
+                  properties:
+                    hive.sql.query SELECT COUNT(*) AS "$f0"
+FROM (SELECT "ss_sold_time_sk", "ss_hdemo_sk", "ss_store_sk"
+FROM "STORE_SALES"
+WHERE "ss_hdemo_sk" IS NOT NULL AND "ss_sold_time_sk" IS NOT NULL AND "ss_store_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "hd_demo_sk"
+FROM "HOUSEHOLD_DEMOGRAPHICS"
+WHERE "hd_dep_count" IN (4, 2, 0) AND ("hd_vehicle_count" <= 5 OR "hd_vehicle_count" <= 2 OR "hd_vehicle_count" <= 3) AND ("hd_dep_count" = 4 AND "hd_vehicle_count" <= 5 OR "hd_dep_count" = 2 AND "hd_vehicle_count" <= 2 OR "hd_dep_count" = 0 AND "hd_vehicle_count" <= 3) AND "hd_demo_sk" IS NOT NULL) AS "t2" ON "t0"."ss_hdemo_sk" = "t2"."hd_demo_sk"
+INNER JOIN (SELECT "t_time_sk"
+FROM "TIME_DIM"
+WHERE "t_hour" = 11 AND "t_minute" < 30 AND "t_time_sk" IS NOT NULL) AS "t4" ON "t0"."ss_sold_time_sk" = "t4"."t_time_sk"
+INNER JOIN (SELECT "s_store_sk"
+FROM "STORE"
+WHERE "s_store_name" = 'ese' AND "s_store_sk" IS NOT NULL) AS "t6" ON "t0"."ss_store_sk" = "t6"."s_store_sk"
+                    hive.sql.query.fieldNames $f0
+                    hive.sql.query.fieldTypes bigint
+                    hive.sql.query.split false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: $f0 (type: bigint)
+                    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)
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Map 14 
+            Map Operator Tree:
+                TableScan
+                  alias: store_sales
+                  properties:
+                    hive.sql.query SELECT COUNT(*) AS "$f0"
+FROM (SELECT "ss_sold_time_sk", "ss_hdemo_sk", "ss_store_sk"
+FROM "STORE_SALES"
+WHERE "ss_hdemo_sk" IS NOT NULL AND "ss_sold_time_sk" IS NOT NULL AND "ss_store_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "hd_demo_sk"
+FROM "HOUSEHOLD_DEMOGRAPHICS"
+WHERE "hd_dep_count" IN (4, 2, 0) AND ("hd_vehicle_count" <= 5 OR "hd_vehicle_count" <= 2 OR "hd_vehicle_count" <= 3) AND ("hd_dep_count" = 4 AND "hd_vehicle_count" <= 5 OR "hd_dep_count" = 2 AND "hd_vehicle_count" <= 2 OR "hd_dep_count" = 0 AND "hd_vehicle_count" <= 3) AND "hd_demo_sk" IS NOT NULL) AS "t2" ON "t0"."ss_hdemo_sk" = "t2"."hd_demo_sk"
+INNER JOIN (SELECT "t_time_sk"
+FROM "TIME_DIM"
+WHERE "t_hour" = 11 AND "t_minute" >= 30 AND "t_time_sk" IS NOT NULL) AS "t4" ON "t0"."ss_sold_time_sk" = "t4"."t_time_sk"
+INNER JOIN (SELECT "s_store_sk"
+FROM "STORE"
+WHERE "s_store_name" = 'ese' AND "s_store_sk" IS NOT NULL) AS "t6" ON "t0"."ss_store_sk" = "t6"."s_store_sk"
+                    hive.sql.query.fieldNames $f0
+                    hive.sql.query.fieldTypes bigint
+                    hive.sql.query.split false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: $f0 (type: bigint)
+                    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)
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Map 15 
+            Map Operator Tree:
+                TableScan
+                  alias: store_sales
+                  properties:
+                    hive.sql.query SELECT COUNT(*) AS "$f0"
+FROM (SELECT "ss_sold_time_sk", "ss_hdemo_sk", "ss_store_sk"
+FROM "STORE_SALES"
+WHERE "ss_hdemo_sk" IS NOT NULL AND "ss_sold_time_sk" IS NOT NULL AND "ss_store_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "hd_demo_sk"
+FROM "HOUSEHOLD_DEMOGRAPHICS"
+WHERE "hd_dep_count" IN (4, 2, 0) AND ("hd_vehicle_count" <= 5 OR "hd_vehicle_count" <= 2 OR "hd_vehicle_count" <= 3) AND ("hd_dep_count" = 4 AND "hd_vehicle_count" <= 5 OR "hd_dep_count" = 2 AND "hd_vehicle_count" <= 2 OR "hd_dep_count" = 0 AND "hd_vehicle_count" <= 3) AND "hd_demo_sk" IS NOT NULL) AS "t2" ON "t0"."ss_hdemo_sk" = "t2"."hd_demo_sk"
+INNER JOIN (SELECT "t_time_sk"
+FROM "TIME_DIM"
+WHERE "t_hour" = 12 AND "t_minute" < 30 AND "t_time_sk" IS NOT NULL) AS "t4" ON "t0"."ss_sold_time_sk" = "t4"."t_time_sk"
+INNER JOIN (SELECT "s_store_sk"
+FROM "STORE"
+WHERE "s_store_name" = 'ese' AND "s_store_sk" IS NOT NULL) AS "t6" ON "t0"."ss_store_sk" = "t6"."s_store_sk"
+                    hive.sql.query.fieldNames $f0
+                    hive.sql.query.fieldTypes bigint
+                    hive.sql.query.split false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: $f0 (type: bigint)
+                    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)
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Map 9 
+            Map Operator Tree:
+                TableScan
+                  alias: store_sales
+                  properties:
+                    hive.sql.query SELECT COUNT(*) AS "$f0"
+FROM (SELECT "ss_sold_time_sk", "ss_hdemo_sk", "ss_store_sk"
+FROM "STORE_SALES"
+WHERE "ss_hdemo_sk" IS NOT NULL AND "ss_sold_time_sk" IS NOT NULL AND "ss_store_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "hd_demo_sk"
+FROM "HOUSEHOLD_DEMOGRAPHICS"
+WHERE "hd_dep_count" IN (4, 2, 0) AND ("hd_vehicle_count" <= 5 OR "hd_vehicle_count" <= 2 OR "hd_vehicle_count" <= 3) AND ("hd_dep_count" = 4 AND "hd_vehicle_count" <= 5 OR "hd_dep_count" = 2 AND "hd_vehicle_count" <= 2 OR "hd_dep_count" = 0 AND "hd_vehicle_count" <= 3) AND "hd_demo_sk" IS NOT NULL) AS "t2" ON "t0"."ss_hdemo_sk" = "t2"."hd_demo_sk"
+INNER JOIN (SELECT "t_time_sk"
+FROM "TIME_DIM"
+WHERE "t_hour" = 9 AND "t_minute" < 30 AND "t_time_sk" IS NOT NULL) AS "t4" ON "t0"."ss_sold_time_sk" = "t4"."t_time_sk"
+INNER JOIN (SELECT "s_store_sk"
+FROM "STORE"
+WHERE "s_store_name" = 'ese' AND "s_store_sk" IS NOT NULL) AS "t6" ON "t0"."ss_store_sk" = "t6"."s_store_sk"
+                    hive.sql.query.fieldNames $f0
+                    hive.sql.query.fieldTypes bigint
+                    hive.sql.query.split false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: $f0 (type: bigint)
+                    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)
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Reducer 2 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 
+                  1 
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1 Data size: 17 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 17 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col0 (type: bigint), _col1 (type: bigint)
+        Reducer 3 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 
+                  1 
+                outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 26 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 26 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: bigint)
+        Reducer 4 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 
+                  1 
+                outputColumnNames: _col0, _col1, _col2, _col3
+                Statistics: Num rows: 1 Data size: 35 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 35 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: bigint), _col3 (type: bigint)
+        Reducer 5 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 
+                  1 
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                Statistics: Num rows: 1 Data size: 44 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 44 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: bigint)
+        Reducer 6 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 
+                  1 
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+                Statistics: Num rows: 1 Data size: 53 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 53 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: bigint), _col5 (type: bigint)
+        Reducer 7 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 
+                  1 
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
+                Statistics: Num rows: 1 Data size: 62 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 62 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col0 (type: bigint), _col1 (type: bigint), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: bigint), _col5 (type: bigint), _col6 (type: bigint)
+        Reducer 8 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 
+                  1 
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
+                Statistics: Num rows: 1 Data size: 71 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 71 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+Warning: Shuffle Join MERGEJOIN[40][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+Warning: Shuffle Join MERGEJOIN[41][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+Warning: Shuffle Join MERGEJOIN[42][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product
+Warning: Shuffle Join MERGEJOIN[43][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4]] in Stage 'Reducer 5' is a cross product
+Warning: Shuffle Join MERGEJOIN[44][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5]] in Stage 'Reducer 6' is a cross product
+Warning: Shuffle Join MERGEJOIN[45][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5, $hdt$_6]] in Stage 'Reducer 7' is a cross product
+Warning: Shuffle Join MERGEJOIN[46][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3, $hdt$_4, $hdt$_5, $hdt$_6, $hdt$_7]] in Stage 'Reducer 8' is a cross product
+PREHOOK: query: select *
+                          from (select count(*) as h8_30_to_9
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 8
+                                  and time_dim.t_minute >= 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s1 join
+                               (select count(*) as h9_to_9_30
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 9
+                                  and time_dim.t_minute < 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s2 join
+                               (select count(*) as h9_30_to_10
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 9
+                                  and time_dim.t_minute >= 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s3 join
+                               (select count(*) as h10_to_10_30
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 10
+                                  and time_dim.t_minute < 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s4 join
+                               (select count(*) as h10_30_to_11
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 10
+                                  and time_dim.t_minute >= 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s5 join
+                               (select count(*) as h11_to_11_30
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 11
+                                  and time_dim.t_minute < 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s6 join
+                               (select count(*) as h11_30_to_12
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 11
+                                  and time_dim.t_minute >= 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s7 join
+                               (select count(*) as h12_to_12_30
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 12
+                                  and time_dim.t_minute < 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s8
+PREHOOK: type: QUERY
+PREHOOK: Input: default@household_demographics
+PREHOOK: Input: default@store
+PREHOOK: Input: default@store_sales
+PREHOOK: Input: default@time_dim
+#### A masked pattern was here ####
+POSTHOOK: query: select *
+                          from (select count(*) as h8_30_to_9
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 8
+                                  and time_dim.t_minute >= 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s1 join
+                               (select count(*) as h9_to_9_30
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 9
+                                  and time_dim.t_minute < 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s2 join
+                               (select count(*) as h9_30_to_10
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 9
+                                  and time_dim.t_minute >= 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s3 join
+                               (select count(*) as h10_to_10_30
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 10
+                                  and time_dim.t_minute < 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s4 join
+                               (select count(*) as h10_30_to_11
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 10
+                                  and time_dim.t_minute >= 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s5 join
+                               (select count(*) as h11_to_11_30
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 11
+                                  and time_dim.t_minute < 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s6 join
+                               (select count(*) as h11_30_to_12
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 11
+                                  and time_dim.t_minute >= 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s7 join
+                               (select count(*) as h12_to_12_30
+                                from store_sales
+                                join household_demographics
+                                  on store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+                                join time_dim
+                                  on store_sales.ss_sold_time_sk = time_dim.t_time_sk
+                                join store
+                                  on store_sales.ss_store_sk = store.s_store_sk
+                                where time_dim.t_hour = 12
+                                  and time_dim.t_minute < 30
+                                  and ((household_demographics.hd_dep_count = 4
+                                        and household_demographics.hd_vehicle_count <= 3+2)
+                                       or (household_demographics.hd_dep_count = 2
+                                           and household_demographics.hd_vehicle_count<=0+2)
+                                       or (household_demographics.hd_dep_count = 0
+                                           and household_demographics.hd_vehicle_count<=1+2))
+                                  and store.s_store_name = 'ese') s8
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@household_demographics
+POSTHOOK: Input: default@store
+POSTHOOK: Input: default@store_sales
+POSTHOOK: Input: default@time_dim
+#### A masked pattern was here ####
+0	0	0	0	0	0	0	0
+PREHOOK: query: explain
+select substr(r_reason_desc, 1, 20) as r,
+avg(ws_quantity) as wq,
+avg(wr_refunded_cash) as ref,
+avg(wr_fee) as fee
+from web_sales
+join web_returns
+on web_sales.ws_item_sk = web_returns.wr_item_sk
+and web_sales.ws_order_number = web_returns.wr_order_number
+join customer_demographics cd1
+on cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk
+join customer_demographics cd2
+on cd2.cd_demo_sk = web_returns.wr_returning_cdemo_sk
+join customer_address
+on customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk
+join date_dim
+on web_sales.ws_sold_date_sk = date_dim.d_date_sk
+join reason
+on reason.r_reason_sk = web_returns.wr_reason_sk
+where d_year = 2000
+and ((cd1.cd_marital_status = 'M'
+and cd1.cd_marital_status = cd2.cd_marital_status
+and cd1.cd_education_status = 'Advanced Degree'
+and cd1.cd_education_status = cd2.cd_education_status
+and ws_sales_price between 100.00 and 150.00)
+or
+(cd1.cd_marital_status = 'S'
+and cd1.cd_marital_status = cd2.cd_marital_status
+and cd1.cd_education_status = 'College'
+and cd1.cd_education_status = cd2.cd_education_status
+and ws_sales_price between 50.00 and 100.00)
+or
+(cd1.cd_marital_status = 'W'
+and cd1.cd_marital_status = cd2.cd_marital_status
+and cd1.cd_education_status = '2 yr Degree'
+and cd1.cd_education_status = cd2.cd_education_status
+and ws_sales_price between 150.00 and 200.00))
+and ((ca_country = 'United States'
+and ca_state in ('IN', 'OH', 'NJ')
+and ws_net_profit between 100 and 200)
+or
+(ca_country = 'United States'
+and ca_state in ('WI', 'CT', 'KY')
+and ws_net_profit between 150 and 300)
+or
+(ca_country = 'United States'
+and ca_state in ('LA', 'IA', 'AR')
+and ws_net_profit between 50 and 250))
+group by r_reason_desc
+order by r, wq, ref, fee
+limit 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@customer_address
+PREHOOK: Input: default@customer_demographics
+PREHOOK: Input: default@date_dim
+PREHOOK: Input: default@reason
+PREHOOK: Input: default@web_returns
+PREHOOK: Input: default@web_sales
+#### A masked pattern was here ####
+POSTHOOK: query: explain
+select substr(r_reason_desc, 1, 20) as r,
+avg(ws_quantity) as wq,
+avg(wr_refunded_cash) as ref,
+avg(wr_fee) as fee
+from web_sales
+join web_returns
+on web_sales.ws_item_sk = web_returns.wr_item_sk
+and web_sales.ws_order_number = web_returns.wr_order_number
+join customer_demographics cd1
+on cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk
+join customer_demographics cd2
+on cd2.cd_demo_sk = web_returns.wr_returning_cdemo_sk
+join customer_address
+on customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk
+join date_dim
+on web_sales.ws_sold_date_sk = date_dim.d_date_sk
+join reason
+on reason.r_reason_sk = web_returns.wr_reason_sk
+where d_year = 2000
+and ((cd1.cd_marital_status = 'M'
+and cd1.cd_marital_status = cd2.cd_marital_status
+and cd1.cd_education_status = 'Advanced Degree'
+and cd1.cd_education_status = cd2.cd_education_status
+and ws_sales_price between 100.00 and 150.00)
+or
+(cd1.cd_marital_status = 'S'
+and cd1.cd_marital_status = cd2.cd_marital_status
+and cd1.cd_education_status = 'College'
+and cd1.cd_education_status = cd2.cd_education_status
+and ws_sales_price between 50.00 and 100.00)
+or
+(cd1.cd_marital_status = 'W'
+and cd1.cd_marital_status = cd2.cd_marital_status
+and cd1.cd_education_status = '2 yr Degree'
+and cd1.cd_education_status = cd2.cd_education_status
+and ws_sales_price between 150.00 and 200.00))
+and ((ca_country = 'United States'
+and ca_state in ('IN', 'OH', 'NJ')
+and ws_net_profit between 100 and 200)
+or
+(ca_country = 'United States'
+and ca_state in ('WI', 'CT', 'KY')
+and ws_net_profit between 150 and 300)
+or
+(ca_country = 'United States'
+and ca_state in ('LA', 'IA', 'AR')
+and ws_net_profit between 50 and 250))
+group by r_reason_desc
+order by r, wq, ref, fee
+limit 100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@customer_address
+POSTHOOK: Input: default@customer_demographics
+POSTHOOK: Input: default@date_dim
+POSTHOOK: Input: default@reason
+POSTHOOK: Input: default@web_returns
+POSTHOOK: Input: default@web_sales
+#### A masked pattern was here ####
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: web_sales
+                  properties:
+                    hive.sql.query SELECT "t12"."r_reason_desc", SUM("t0"."ws_quantity") AS "$f1", COUNT("t0"."ws_quantity") AS "$f2", SUM("t2"."wr_refunded_cash") AS "$f3", COUNT("t2"."wr_refunded_cash") AS "$f4", SUM("t2"."wr_fee") AS "$f5", COUNT("t2"."wr_fee") AS "$f6"
+FROM (SELECT "ws_sold_date_sk", "ws_item_sk", "ws_order_number", "ws_quantity", "ws_net_profit" BETWEEN 100 AND 200 AS "BETWEEN", "ws_net_profit" BETWEEN 150 AND 300 AS "BETWEEN5", "ws_net_profit" BETWEEN 50 AND 250 AS "BETWEEN6", "ws_sales_price" BETWEEN 100 AND 150 AS "BETWEEN7", "ws_sales_price" BETWEEN 50 AND 100 AS "BETWEEN8", "ws_sales_price" BETWEEN 150 AND 200 AS "BETWEEN9"
+FROM "WEB_SALES"
+WHERE (100 <= "ws_sales_price" OR "ws_sales_price" <= 150 OR 50 <= "ws_sales_price" OR "ws_sales_price" <= 100 OR 150 <= "ws_sales_price" OR "ws_sales_price" <= 200) AND (100 <= "ws_net_profit" OR "ws_net_profit" <= 200 OR 150 <= "ws_net_profit" OR "ws_net_profit" <= 300 OR 50 <= "ws_net_profit" OR "ws_net_profit" <= 250) AND "ws_item_sk" IS NOT NULL AND "ws_order_number" IS NOT NULL AND "ws_sold_date_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "wr_item_sk", "wr_refunded_cdemo_sk", "wr_refunded_addr_sk", "wr_returning_cdemo_sk", "wr_reason_sk", "wr_order_number", "wr_fee", "wr_refunded_cash"
+FROM "WEB_RETURNS"
+WHERE "wr_item_sk" IS NOT NULL AND "wr_order_number" IS NOT NULL AND "wr_refunded_cdemo_sk" IS NOT NULL AND "wr_returning_cdemo_sk" IS NOT NULL AND "wr_refunded_addr_sk" IS NOT NULL AND "wr_reason_sk" IS NOT NULL) AS "t2" ON "t0"."ws_item_sk" = "t2"."wr_item_sk" AND "t0"."ws_order_number" = "t2"."wr_order_number"
+INNER JOIN (SELECT "cd_demo_sk", "cd_marital_status", "cd_education_status", "cd_marital_status" = 'M' AS "=", "cd_education_status" = 'Advanced Degree     ' AS "=4", "cd_marital_status" = 'S' AS "=5", "cd_education_status" = 'College             ' AS "=6", "cd_marital_status" = 'W' AS "=7", "cd_education_status" = '2 yr Degree         ' AS "=8"
+FROM "CUSTOMER_DEMOGRAPHICS"
+WHERE "cd_marital_status" IN ('M', 'S', 'W') AND "cd_education_status" IN ('Advanced Degree     ', 'College             ', '2 yr Degree         ') AND "cd_demo_sk" IS NOT NULL) AS "t4" ON "t2"."wr_refunded_cdemo_sk" = "t4"."cd_demo_sk" AND ("t4"."=" AND "t4"."=4" AND "t0"."BETWEEN7" OR "t4"."=5" AND "t4"."=6" AND "t0"."BETWEEN8" OR "t4"."=7" AND "t4"."=8" AND "t0"."BETWEEN9")
+INNER JOIN (SELECT "cd_demo_sk", "cd_marital_status", "cd_education_status"
+FROM "CUSTOMER_DEMOGRAPHICS"
+WHERE "cd_marital_status" IN ('M', 'S', 'W') AND "cd_education_status" IN ('Advanced Degree     ', 'College             ', '2 yr Degree         ') AND "cd_demo_sk" IS NOT NULL) AS "t6" ON "t2"."wr_returning_cdemo_sk" = "t6"."cd_demo_sk" AND "t4"."cd_marital_status" = "t6"."cd_marital_status" AND "t4"."cd_education_status" = "t6"."cd_education_status"
+INNER JOIN (SELECT "ca_address_sk", "ca_state" IN ('IN', 'OH', 'NJ') AS "IN", "ca_state" IN ('WI', 'CT', 'KY') AS "IN2", "ca_state" IN ('LA', 'IA', 'AR') AS "IN3"
+FROM "CUSTOMER_ADDRESS"
+WHERE "ca_state" IN ('IN', 'OH', 'NJ', 'WI', 'CT', 'KY', 'LA', 'IA', 'AR') AND "ca_country" = 'United States' AND "ca_address_sk" IS NOT NULL) AS "t8" ON "t2"."wr_refunded_addr_sk" = "t8"."ca_address_sk" AND ("t8"."IN" AND "t0"."BETWEEN" OR "t8"."IN2" AND "t0"."BETWEEN5" OR "t8"."IN3" AND "t0"."BETWEEN6")
+INNER JOIN (SELECT "d_date_sk"
+FROM "DATE_DIM"
+WHERE "d_year" = 2000 AND "d_date_sk" IS NOT NULL) AS "t10" ON "t0"."ws_sold_date_sk" = "t10"."d_date_sk"
+INNER JOIN (SELECT "r_reason_sk", "r_reason_desc"
+FROM "REASON"
+WHERE "r_reason_sk" IS NOT NULL) AS "t12" ON "t2"."wr_reason_sk" = "t12"."r_reason_sk"
+GROUP BY "t12"."r_reason_desc"
+                    hive.sql.query.fieldNames r_reason_desc,$f1,$f2,$f3,$f4,$f5,$f6
+                    hive.sql.query.fieldTypes char(100),bigint,bigint,decimal(11,6),bigint,decimal(11,6),bigint
+                    hive.sql.query.split false
+                  Statistics: Num rows: 1 Data size: 440 Basic stats: COMPLETE Column stats: NONE
+                  Select Operator
+                    expressions: substr(r_reason_desc, 1, 20) (type: string), (UDFToDouble($f1) / $f2) (type: double), ($f3 / $f4) (type: decimal(31,26)), ($f5 / $f6) (type: decimal(31,26))
+                    outputColumnNames: _col0, _col1, _col2, _col3
+                    Statistics: Num rows: 1 Data size: 440 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: string), _col1 (type: double), _col2 (type: decimal(31,26)), _col3 (type: decimal(31,26))
+                      sort order: ++++
+                      Statistics: Num rows: 1 Data size: 440 Basic stats: COMPLETE Column stats: NONE
+                      TopN Hash Memory Usage: 0.1
+            Execution mode: vectorized, llap
+            LLAP IO: no inputs
+        Reducer 2 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: double), KEY.reducesinkkey2 (type: decimal(31,26)), KEY.reducesinkkey3 (type: decimal(31,26))
+                outputColumnNames: _col0, _col1, _col2, _col3
+                Statistics: Num rows: 1 Data size: 440 Basic stats: COMPLETE Column stats: NONE
+                Limit
+                  Number of rows: 100
+                  Statistics: Num rows: 1 Data size: 440 Basic stats: COMPLETE Column stats: NONE
+                  File Output Operator
+                    compressed: false
+                    Statistics: Num rows: 1 Data size: 440 Basic stats: COMPLETE Column stats: NONE
+                    table:
+                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: 100
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select substr(r_reason_desc, 1, 20) as r,
+avg(ws_quantity) as wq,
+avg(wr_refunded_cash) as ref,
+avg(wr_fee) as fee
+from web_sales
+join web_returns
+on web_sales.ws_item_sk = web_returns.wr_item_sk
+and web_sales.ws_order_number = web_returns.wr_order_number
+join customer_demographics cd1
+on cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk
+join customer_demographics cd2
+on cd2.cd_demo_sk = web_returns.wr_returning_cdemo_sk
+join customer_address
+on customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk
+join date_dim
+on web_sales.ws_sold_date_sk = date_dim.d_date_sk
+join reason
+on reason.r_reason_sk = web_returns.wr_reason_sk
+where d_year = 2000
+and ((cd1.cd_marital_status = 'M'
+and cd1.cd_marital_status = cd2.cd_marital_status
+and cd1.cd_education_status = 'Advanced Degree'
+and cd1.cd_education_status = cd2.cd_education_status
+and ws_sales_price between 100.00 and 150.00)
+or
+(cd1.cd_marital_status = 'S'
+and cd1.cd_marital_status = cd2.cd_marital_status
+and cd1.cd_education_status = 'College'
+and cd1.cd_education_status = cd2.cd_education_status
+and ws_sales_price between 50.00 and 100.00)
+or
+(cd1.cd_marital_status = 'W'
+and cd1.cd_marital_status = cd2.cd_marital_status
+and cd1.cd_education_status = '2 yr Degree'
+and cd1.cd_education_status = cd2.cd_education_status
+and ws_sales_price between 150.00 and 200.00))
+and ((ca_country = 'United States'
+and ca_state in ('IN', 'OH', 'NJ')
+and ws_net_profit between 100 and 200)
+or
+(ca_country = 'United States'
+and ca_state in ('WI', 'CT', 'KY')
+and ws_net_profit between 150 and 300)
+or
+(ca_country = 'United States'
+and ca_state in ('LA', 'IA', 'AR')
+and ws_net_profit between 50 and 250))
+group by r_reason_desc
+order by r, wq, ref, fee
+limit 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@customer_address
+PREHOOK: Input: default@customer_demographics
+PREHOOK: Input: default@date_dim
+PREHOOK: Input: default@reason
+PREHOOK: Input: default@web_returns
+PREHOOK: Input: default@web_sales
+#### A masked pattern was here ####
+POSTHOOK: query: select substr(r_reason_desc, 1, 20) as r,
+avg(ws_quantity) as wq,
+avg(wr_refunded_cash) as ref,
+avg(wr_fee) as fee
+from web_sales
+join web_returns
+on web_sales.ws_item_sk = web_returns.wr_item_sk
+and web_sales.ws_order_number = web_returns.wr_order_number
+join customer_demographics cd1
+on cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk
+join customer_demographics cd2
+on cd2.cd_demo_sk = web_returns.wr_returning_cdemo_sk
+join customer_address
+on customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk
+join date_dim
+on web_sales.ws_sold_date_sk = date_dim.d_date_sk
+join reason
+on reason.r_reason_sk = web_returns.wr_reason_sk
+where d_year = 2000
+and ((cd1.cd_marital_status = 'M'
+and cd1.cd_marital_status = cd2.cd_marital_status
+and cd1.cd_education_status = 'Advanced Degree'
+and cd1.cd_education_status = cd2.cd_education_status
+and ws_sales_price between 100.00 and 150.00)
+or
+(cd1.cd_marital_status = 'S'
+and cd1.cd_marital_status = cd2.cd_marital_status
+and cd1.cd_education_status = 'College'
+and cd1.cd_education_status = cd2.cd_education_status
+and ws_sales_price between 50.00 and 100.00)
+or
+(cd1.cd_marital_status = 'W'
+and cd1.cd_marital_status = cd2.cd_marital_status
+and cd1.cd_education_status = '2 yr Degree'
+and cd1.cd_education_status = cd2.cd_education_status
+and ws_sales_price between 150.00 and 200.00))
+and ((ca_country = 'United States'
+and ca_state in ('IN', 'OH', 'NJ')
+and ws_net_profit between 100 and 200)
+or
+(ca_country = 'United States'
+and ca_state in ('WI', 'CT', 'KY')
+and ws_net_profit between 150 and 300)
+or
+(ca_country = 'United States'
+and ca_state in ('LA', 'IA', 'AR')
+and ws_net_profit between 50 and 250))
+group by r_reason_desc
+order by r, wq, ref, fee
+limit 100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@customer_address
+POSTHOOK: Input: default@customer_demographics
+POSTHOOK: Input: default@date_dim
+POSTHOOK: Input: default@reason
+POSTHOOK: Input: default@web_returns
+POSTHOOK: Input: default@web_sales
+#### A masked pattern was here ####
+PREHOOK: query: DROP TABLE catalog_sales
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@catalog_sales
+PREHOOK: Output: default@catalog_sales
+POSTHOOK: query: DROP TABLE catalog_sales
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@catalog_sales
+POSTHOOK: Output: default@catalog_sales
+PREHOOK: query: DROP TABLE catalog_returns
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@catalog_returns
+PREHOOK: Output: default@catalog_returns
+POSTHOOK: query: DROP TABLE catalog_returns
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@catalog_returns
+POSTHOOK: Output: default@catalog_returns
+PREHOOK: query: DROP TABLE store_sales
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@store_sales
+PREHOOK: Output: default@store_sales
+POSTHOOK: query: DROP TABLE store_sales
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@store_sales
+POSTHOOK: Output: default@store_sales
+PREHOOK: query: DROP TABLE store_returns
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@store_returns
+PREHOOK: Output: default@store_returns
+POSTHOOK: query: DROP TABLE store_returns
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@store_returns
+POSTHOOK: Output: default@store_returns
+PREHOOK: query: DROP TABLE web_sales
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@web_sales
+PREHOOK: Output: default@web_sales
+POSTHOOK: query: DROP TABLE web_sales
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@web_sales
+POSTHOOK: Output: default@web_sales
+PREHOOK: query: DROP TABLE web_returns
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@web_returns
+PREHOOK: Output: default@web_returns
+POSTHOOK: query: DROP TABLE web_returns
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@web_returns
+POSTHOOK: Output: default@web_returns
+PREHOOK: query: DROP TABLE customer
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@customer
+PREHOOK: Output: default@customer
+POSTHOOK: query: DROP TABLE customer
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@customer
+POSTHOOK: Output: default@customer
+PREHOOK: query: DROP TABLE customer_address
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@customer_address
+PREHOOK: Output: default@customer_address
+POSTHOOK: query: DROP TABLE customer_address
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@customer_address
+POSTHOOK: Output: default@customer_address
+PREHOOK: query: DROP TABLE customer_demographics
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@customer_demographics
+PREHOOK: Output: default@customer_demographics
+POSTHOOK: query: DROP TABLE customer_demographics
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@customer_demographics
+POSTHOOK: Output: default@customer_demographics
+PREHOOK: query: DROP TABLE inventory
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@inventory
+PREHOOK: Output: default@inventory
+POSTHOOK: query: DROP TABLE inventory
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@inventory
+POSTHOOK: Output: default@inventory
+PREHOOK: query: DROP TABLE item
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@item
+PREHOOK: Output: default@item
+POSTHOOK: query: DROP TABLE item
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@item
+POSTHOOK: Output: default@item
+PREHOOK: query: DROP TABLE warehouse
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@warehouse
+PREHOOK: Output: default@warehouse
+POSTHOOK: query: DROP TABLE warehouse
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@warehouse
+POSTHOOK: Output: default@warehouse
+PREHOOK: query: DROP TABLE date_dim
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@date_dim
+PREHOOK: Output: default@date_dim
+POSTHOOK: query: DROP TABLE date_dim
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@date_dim
+POSTHOOK: Output: default@date_dim
+PREHOOK: query: DROP TABLE store
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@store
+PREHOOK: Output: default@store
+POSTHOOK: query: DROP TABLE store
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@store
+POSTHOOK: Output: default@store
+PREHOOK: query: DROP TABLE household_demographics
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@household_demographics
+PREHOOK: Output: default@household_demographics
+POSTHOOK: query: DROP TABLE household_demographics
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@household_demographics
+POSTHOOK: Output: default@household_demographics
+PREHOOK: query: DROP TABLE time_dim
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@time_dim
+PREHOOK: Output: default@time_dim
+POSTHOOK: query: DROP TABLE time_dim
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@time_dim
+POSTHOOK: Output: default@time_dim
+PREHOOK: query: DROP TABLE reason
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@reason
+PREHOOK: Output: default@reason
+POSTHOOK: query: DROP TABLE reason
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@reason
+POSTHOOK: Output: default@reason
+PREHOOK: query: FROM src
+SELECT
+#### A masked pattern was here ####
+'DROP TABLE CATALOG_SALES' ),
+#### A masked pattern was here ####
+'DROP TABLE CATALOG_RETURNS' ),
+#### A masked pattern was here ####
+'DROP TABLE STORE_SALES' ),
+#### A masked pattern was here ####
+'DROP TABLE STORE_RETURNS' ),
+#### A masked pattern was here ####
+'DROP TABLE WEB_SALES' ),
+#### A masked pattern was here ####
+'DROP TABLE WEB_RETURNS' ),
+#### A masked pattern was here ####
+'DROP TABLE CUSTOMER' ),
+#### A masked pattern was here ####
+'DROP TABLE CUSTOMER_ADDRESS' ),
+#### A masked pattern was here ####
+'DROP TABLE CUSTOMER_DEMOGRAPHICS' ),
+#### A masked pattern was here ####
+'DROP TABLE INVENTORY' ),
+#### A masked pattern was here ####
+'DROP TABLE ITEM' ),
+#### A masked pattern was here ####
+'DROP TABLE WAREHOUSE' ),
+#### A masked pattern was here ####
+'DROP TABLE DATE_DIM' ),
+#### A masked pattern was here ####
+'DROP TABLE STORE' ),
+#### A masked pattern was here ####
+'DROP TABLE HOUSEHOLD_DEMOGRAPHICS' ),
+#### A masked pattern was here ####
+'DROP TABLE TIME_DIM' ),
+#### A masked pattern was here ####
+'DROP TABLE REASON' )
+limit 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: FROM src
+SELECT
+#### A masked pattern was here ####
+'DROP TABLE CATALOG_SALES' ),
+#### A masked pattern was here ####
+'DROP TABLE CATALOG_RETURNS' ),
+#### A masked pattern was here ####
+'DROP TABLE STORE_SALES' ),
+#### A masked pattern was here ####
+'DROP TABLE STORE_RETURNS' ),
+#### A masked pattern was here ####
+'DROP TABLE WEB_SALES' ),
+#### A masked pattern was here ####
+'DROP TABLE WEB_RETURNS' ),
+#### A masked pattern was here ####
+'DROP TABLE CUSTOMER' ),
+#### A masked pattern was here ####
+'DROP TABLE CUSTOMER_ADDRESS' ),
+#### A masked pattern was here ####
+'DROP TABLE CUSTOMER_DEMOGRAPHICS' ),
+#### A masked pattern was here ####
+'DROP TABLE INVENTORY' ),
+#### A masked pattern was here ####
+'DROP TABLE ITEM' ),
+#### A masked pattern was here ####
+'DROP TABLE WAREHOUSE' ),
+#### A masked pattern was here ####
+'DROP TABLE DATE_DIM' ),
+#### A masked pattern was here ####
+'DROP TABLE STORE' ),
+#### A masked pattern was here ####
+'DROP TABLE HOUSEHOLD_DEMOGRAPHICS' ),
+#### A masked pattern was here ####
+'DROP TABLE TIME_DIM' ),
+#### A masked pattern was here ####
+'DROP TABLE REASON' )
+limit 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+0	0	0	0	0	0	0	0	0	0	0	0	0	0	0	0	0
diff --git a/ql/src/test/results/clientpositive/llap/sharedwork.q.out b/ql/src/test/results/clientpositive/llap/sharedwork.q.out
index 3721a0a..f46c132 100644
--- a/ql/src/test/results/clientpositive/llap/sharedwork.q.out
+++ b/ql/src/test/results/clientpositive/llap/sharedwork.q.out
@@ -100,6 +100,22 @@ POSTHOOK: Input: default@my_table_0001_00
 POSTHOOK: Input: default@my_table_0001_01
 POSTHOOK: Input: default@my_table_0003
 #### A masked pattern was here ####
+OPTIMIZED SQL: SELECT `t0`.`col_7`, `t0`.`CAST` AS `col_3`, `t0`.`col_20`, `t2`.`col_21` AS `col_21_1232`, `t0`.`col_1`, `t4`.`col_22`, `t6`.`col_21` AS `col_21_879`, `t4`.`col_23`
+FROM (SELECT `col_1`, `col_7`, `col_20`, CAST(`col_3` AS DATE) AS `CAST`
+FROM `default`.`my_table_0001`
+WHERE `col_20` IN ('part1', 'part2', 'part3') AND (CAST(`col_7` AS DOUBLE) IS NOT NULL OR `col_7` IS NULL) AND CAST(`col_3` AS DATE) BETWEEN DATE '2018-07-01' AND DATE '2019-01-23') AS `t0`
+LEFT JOIN (SELECT `col_24`, `col_21`
+FROM `default`.`my_table_0003`
+WHERE `col_24` IN ('part1', 'part2', 'part3')) AS `t2` ON `t0`.`col_20` = `t2`.`col_24`
+LEFT JOIN (SELECT `col_1`, `col_22`, `col_23`
+FROM `default`.`my_table_0001_00`
+WHERE `col_1` IS NOT NULL) AS `t4` ON `t0`.`col_1` = `t4`.`col_1`
+LEFT JOIN (SELECT `col_24`, `col_21`
+FROM `default`.`my_table_0003`
+WHERE `col_24` IS NOT NULL) AS `t6` ON `t4`.`col_22` = `t6`.`col_24`
+LEFT JOIN (SELECT `col_1`
+FROM `default`.`my_table_0001_01`
+WHERE `col_100` = 210 AND `col_1` IS NOT NULL) AS `t8` ON `t0`.`col_1` = `t8`.`col_1`
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
   Stage-0 depends on stages: Stage-1
diff --git a/ql/src/test/results/clientpositive/pcr.q.out b/ql/src/test/results/clientpositive/pcr.q.out
index 772e62e..f3fb02c 100644
--- a/ql/src/test/results/clientpositive/pcr.q.out
+++ b/ql/src/test/results/clientpositive/pcr.q.out
@@ -3068,6 +3068,10 @@ POSTHOOK: Input: default@pcr_t1@ds=2000-04-09
 POSTHOOK: Input: default@pcr_t1@ds=2000-04-10
 POSTHOOK: Input: default@pcr_t1@ds=2000-04-11
 #### A masked pattern was here ####
+OPTIMIZED SQL: SELECT `key`, `value`, `ds`
+FROM `default`.`pcr_t1`
+WHERE `ds` > '2000-04-08' AND `ds` < '2000-04-11' OR `ds` BETWEEN '2000-04-08' AND '2000-04-11' AND `key` = 2
+ORDER BY `key`, `value`, `ds`
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
   Stage-0 depends on stages: Stage-1
diff --git a/ql/src/test/results/clientpositive/rand_partitionpruner3.q.out b/ql/src/test/results/clientpositive/rand_partitionpruner3.q.out
index 5e19d87..a394efa 100644
--- a/ql/src/test/results/clientpositive/rand_partitionpruner3.q.out
+++ b/ql/src/test/results/clientpositive/rand_partitionpruner3.q.out
@@ -8,6 +8,9 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@srcpart
 POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
 #### A masked pattern was here ####
+OPTIMIZED SQL: SELECT `key`, `value`, CAST('2008-04-08' AS STRING) AS `ds`, `hr`
+FROM `default`.`srcpart`
+WHERE RAND(1) < 0.1 AND `ds` = '2008-04-08' AND CAST(`key` AS DOUBLE) BETWEEN 10 AND 50 AND `hr` LIKE '%2'
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
   Stage-0 depends on stages: Stage-1
@@ -137,6 +140,9 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@srcpart
 POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
 #### A masked pattern was here ####
+OPTIMIZED SQL: SELECT `key`, `value`, CAST('2008-04-08' AS STRING) AS `ds`, `hr`
+FROM `default`.`srcpart`
+WHERE `ds` = '2008-04-08' AND CAST(`key` AS DOUBLE) BETWEEN 10 AND 50 AND `hr` LIKE '%2'
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
   Stage-0 depends on stages: Stage-1
diff --git a/ql/src/test/results/clientpositive/spark/join34.q.out b/ql/src/test/results/clientpositive/spark/join34.q.out
index 652f1af..74447de 100644
--- a/ql/src/test/results/clientpositive/spark/join34.q.out
+++ b/ql/src/test/results/clientpositive/spark/join34.q.out
@@ -32,6 +32,17 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src
 POSTHOOK: Input: default@src1
 POSTHOOK: Output: default@dest_j1_n1
+OPTIMIZED SQL: SELECT `t5`.`key`, `t5`.`value`, `t3`.`value` AS `value1`
+FROM (SELECT `key`, `value`
+FROM `default`.`src`
+WHERE `key` < 20 AND CAST(`key` AS DOUBLE) BETWEEN 20 AND 100
+UNION ALL
+SELECT `key`, `value`
+FROM `default`.`src`
+WHERE `key` > 100 AND CAST(`key` AS DOUBLE) BETWEEN 20 AND 100) AS `t3`
+INNER JOIN (SELECT `key`, `value`
+FROM `default`.`src1`
+WHERE CAST(`key` AS DOUBLE) BETWEEN 20 AND 100 AND `key` IS NOT NULL) AS `t5` ON `t3`.`key` = `t5`.`key`
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
   Stage-0 depends on stages: Stage-1
diff --git a/ql/src/test/results/clientpositive/spark/join35.q.out b/ql/src/test/results/clientpositive/spark/join35.q.out
index 1fab817..40d5160 100644
--- a/ql/src/test/results/clientpositive/spark/join35.q.out
+++ b/ql/src/test/results/clientpositive/spark/join35.q.out
@@ -32,6 +32,19 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src
 POSTHOOK: Input: default@src1
 POSTHOOK: Output: default@dest_j1_n24
+OPTIMIZED SQL: SELECT `t5`.`key`, `t5`.`value`, `t3`.`$f1` AS `cnt`
+FROM (SELECT `key`, COUNT(*) AS `$f1`
+FROM `default`.`src`
+WHERE `key` < 20 AND CAST(`key` AS DOUBLE) BETWEEN 20 AND 100
+GROUP BY `key`
+UNION ALL
+SELECT `key`, COUNT(*) AS `$f1`
+FROM `default`.`src`
+WHERE `key` > 100 AND CAST(`key` AS DOUBLE) BETWEEN 20 AND 100
+GROUP BY `key`) AS `t3`
+INNER JOIN (SELECT `key`, `value`
+FROM `default`.`src1`
+WHERE CAST(`key` AS DOUBLE) BETWEEN 20 AND 100 AND `key` IS NOT NULL) AS `t5` ON `t3`.`key` = `t5`.`key`
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
   Stage-0 depends on stages: Stage-1
diff --git a/ql/src/test/results/clientpositive/spark/pcr.q.out b/ql/src/test/results/clientpositive/spark/pcr.q.out
index b12730c..3523929 100644
--- a/ql/src/test/results/clientpositive/spark/pcr.q.out
+++ b/ql/src/test/results/clientpositive/spark/pcr.q.out
@@ -3115,6 +3115,10 @@ POSTHOOK: Input: default@pcr_t1@ds=2000-04-09
 POSTHOOK: Input: default@pcr_t1@ds=2000-04-10
 POSTHOOK: Input: default@pcr_t1@ds=2000-04-11
 #### A masked pattern was here ####
+OPTIMIZED SQL: SELECT `key`, `value`, `ds`
+FROM `default`.`pcr_t1`
+WHERE `ds` > '2000-04-08' AND `ds` < '2000-04-11' OR `ds` BETWEEN '2000-04-08' AND '2000-04-11' AND `key` = 2
+ORDER BY `key`, `value`, `ds`
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
   Stage-0 depends on stages: Stage-1