HIVE-20296: Improve HivePointLookupOptimizerRule to be able to extract from 
more sophisticated contexts (Zoltan Haindrich reviewed by Ashutosh Chauhan)

Signed-off-by: Zoltan Haindrich <k...@rxd.hu>


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/93de9504
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/93de9504
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/93de9504

Branch: refs/heads/master
Commit: 93de95041231aa82f7837f254f80a231f5c8d36e
Parents: 86685c6
Author: Zoltan Haindrich <k...@rxd.hu>
Authored: Tue Sep 11 13:06:53 2018 +0200
Committer: Zoltan Haindrich <k...@rxd.hu>
Committed: Tue Sep 11 13:06:53 2018 +0200

----------------------------------------------------------------------
 .../rules/HivePointLookupOptimizerRule.java     | 309 ++++++++++++-------
 .../rules/TestHivePointLookupOptimizerRule.java | 174 +++++++++++
 .../druid/druidmini_test_ts.q.out               |   2 +-
 .../clientpositive/llap/bucketpruning1.q.out    |   7 +-
 .../clientpositive/perf/spark/query15.q.out     |   2 +-
 .../clientpositive/perf/spark/query47.q.out     |  30 +-
 .../clientpositive/perf/spark/query57.q.out     |  30 +-
 .../clientpositive/perf/tez/query15.q.out       |   2 +-
 .../clientpositive/perf/tez/query47.q.out       |   8 +-
 .../clientpositive/perf/tez/query57.q.out       |   8 +-
 10 files changed, 412 insertions(+), 160 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/93de9504/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HivePointLookupOptimizerRule.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HivePointLookupOptimizerRule.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HivePointLookupOptimizerRule.java
index eff9a31..4d42ab4 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HivePointLookupOptimizerRule.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HivePointLookupOptimizerRule.java
@@ -19,6 +19,8 @@ package org.apache.hadoop.hive.ql.optimizer.calcite.rules;
 
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -44,22 +46,19 @@ import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveIn;
-import org.apache.hadoop.hive.ql.optimizer.calcite.translator.TypeConverter;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.collect.ArrayListMultimap;
+import com.google.common.base.Function;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.LinkedHashMultimap;
-import com.google.common.collect.ListMultimap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Multimap;
+import com.google.common.collect.Multimaps;
 import com.google.common.collect.Sets;
 
-
 public abstract class HivePointLookupOptimizerRule extends RelOptRule {
 
 /**
@@ -74,6 +73,7 @@ public abstract class HivePointLookupOptimizerRule extends 
RelOptRule {
       super(operand(Filter.class, any()), minNumORClauses);
     }
 
+    @Override
     public void onMatch(RelOptRuleCall call) {
       final Filter filter = call.rel(0);
       final RexBuilder rexBuilder = filter.getCluster().getRexBuilder();
@@ -93,12 +93,13 @@ public abstract class HivePointLookupOptimizerRule extends 
RelOptRule {
  * to generate an IN clause (which is more efficient). If the OR operator 
contains
  * AND operator children, the optimization might generate an IN clause that 
uses
  * structs.
- */  
+ */
   public static class JoinCondition extends HivePointLookupOptimizerRule {
     public JoinCondition (int minNumORClauses) {
       super(operand(Join.class, any()), minNumORClauses);
     }
-    
+
+    @Override
     public void onMatch(RelOptRuleCall call) {
       final Join join = call.rel(0);
       final RexBuilder rexBuilder = join.getCluster().getRexBuilder();
@@ -132,7 +133,7 @@ public abstract class HivePointLookupOptimizerRule extends 
RelOptRule {
 
   public void analyzeCondition(RelOptRuleCall call,
           RexBuilder rexBuilder,
-          AbstractRelNode node, 
+          AbstractRelNode node,
           RexNode condition) {
 
     // 1. We try to transform possible candidates
@@ -173,29 +174,31 @@ public abstract class HivePointLookupOptimizerRule 
extends RelOptRule {
     @Override public RexNode visitCall(RexCall call) {
       RexNode node;
       switch (call.getKind()) {
-        case AND:
-          ImmutableList<RexNode> operands = RexUtil.flattenAnd(((RexCall) 
call).getOperands());
-          List<RexNode> newOperands = new ArrayList<RexNode>();
-          for (RexNode operand: operands) {
-            RexNode newOperand;
-            if (operand.getKind() == SqlKind.OR) {
-              try {
-                newOperand = transformIntoInClauseCondition(rexBuilder,
-                        nodeOp.getRowType(), operand, minNumORClauses);
-                if (newOperand == null) {
-                  newOperand = operand;
-                }
-              } catch (SemanticException e) {
-                LOG.error("Exception in HivePointLookupOptimizerRule", e);
-                return call;
+      // FIXME: I don't think there is a need for this right now...calcite 
have already done the flattening/etc
+      // removing this case clause will not miss the OR below AND
+      case AND:
+        ImmutableList<RexNode> operands = 
RexUtil.flattenAnd(call.getOperands());
+        List<RexNode> newOperands = new ArrayList<RexNode>();
+        for (RexNode operand : operands) {
+          RexNode newOperand;
+          if (operand.getKind() == SqlKind.OR) {
+            try {
+              newOperand = transformIntoInClauseCondition(rexBuilder,
+                  nodeOp.getRowType(), operand, minNumORClauses);
+              if (newOperand == null) {
+                newOperand = operand;
               }
-            } else {
-              newOperand = operand;
+            } catch (SemanticException e) {
+              LOG.error("Exception in HivePointLookupOptimizerRule", e);
+              return call;
             }
-            newOperands.add(newOperand);
+          } else {
+            newOperand = operand;
           }
-          node = RexUtil.composeConjunction(rexBuilder, newOperands, false);
-          break;
+          newOperands.add(newOperand);
+        }
+        node = RexUtil.composeConjunction(rexBuilder, newOperands, false);
+        break;
         case OR:
           try {
             node = transformIntoInClauseCondition(rexBuilder,
@@ -214,106 +217,184 @@ public abstract class HivePointLookupOptimizerRule 
extends RelOptRule {
       return node;
     }
 
-    private static RexNode transformIntoInClauseCondition(RexBuilder 
rexBuilder, RelDataType inputSchema,
+    /**
+     * Represents a simple contraint.
+     *
+     * Example: a=1
+     */
+    static class Constraint {
+
+      private RexLiteral literal;
+      private RexInputRef inputRef;
+
+      public Constraint(RexInputRef inputRef, RexLiteral literal) {
+        this.literal = literal;
+        this.inputRef = inputRef;
+      }
+
+      /**
+       * Interprets argument as a constraint; if not possible returns null.
+       */
+      public static Constraint of(RexNode n) {
+        if (!(n instanceof RexCall)) {
+          return null;
+        }
+        RexCall call = (RexCall) n;
+        if (call.getOperator().getKind() != SqlKind.EQUALS) {
+          return null;
+        }
+        RexNode opA = call.operands.get(0);
+        RexNode opB = call.operands.get(1);
+        if (opA instanceof RexLiteral && opB instanceof RexInputRef) {
+          RexLiteral rexLiteral = (RexLiteral) opA;
+          RexInputRef rexInputRef = (RexInputRef) opB;
+          return new Constraint(rexInputRef, rexLiteral);
+        }
+        if (opA instanceof RexInputRef && opB instanceof RexLiteral) {
+          RexLiteral rexLiteral = (RexLiteral) opB;
+          RexInputRef rexInputRef = (RexInputRef) opA;
+          return new Constraint(rexInputRef, rexLiteral);
+        }
+        return null;
+      }
+
+      public RexInputRef getKey() {
+        return inputRef;
+      }
+
+    }
+
+    /**
+     * A group of Constraints.
+     *
+     * Examples:
+     *  (a=1 && b=1)
+     *  (a=1)
+     *
+     * Note: any rexNode is accepted as constraint; but it might be keyed with 
the empty key;
+     * which means it can't be parsed as a constraint for some reason; but for 
completeness...
+     *
+     */
+    static class ConstraintGroup {
+
+      public static final Function<ConstraintGroup, Set<RexInputRef>> 
KEY_FUNCTION = new Function<ConstraintGroup, Set<RexInputRef>>() {
+
+        @Override
+        public Set<RexInputRef> apply(ConstraintGroup a) {
+          return a.key;
+        }
+      };
+      private Map<RexInputRef, Constraint> constraints = new HashMap<>();
+      private RexNode originalRexNode;
+      private final Set<RexInputRef> key;
+
+      public ConstraintGroup(RexNode rexNode) {
+        originalRexNode = rexNode;
+
+        final List<RexNode> conjunctions = RelOptUtil.conjunctions(rexNode);
+
+        for (RexNode n : conjunctions) {
+
+          Constraint c = Constraint.of(n);
+          if (c == null) {
+            // interpretation failed; make this node opaque
+            key = Collections.emptySet();
+            return;
+          }
+          constraints.put(c.getKey(), c);
+        }
+        if (constraints.size() != conjunctions.size()) {
+          LOG.debug("unexpected situation; giving up on this branch");
+          key = Collections.emptySet();
+          return;
+        }
+        key = constraints.keySet();
+      }
+
+      public List<RexNode> getValuesInOrder(List<RexInputRef> columns) throws 
SemanticException {
+        List<RexNode> ret = new ArrayList<>();
+        for (RexInputRef rexInputRef : columns) {
+          Constraint constraint = constraints.get(rexInputRef);
+          if (constraint == null) {
+            throw new SemanticException("Unable to find constraint which was 
earlier added.");
+          }
+          ret.add(constraint.literal);
+        }
+        return ret;
+      }
+    }
+
+    private RexNode transformIntoInClauseCondition(RexBuilder rexBuilder, 
RelDataType inputSchema,
             RexNode condition, int minNumORClauses) throws SemanticException {
       assert condition.getKind() == SqlKind.OR;
 
-      // 1. We extract the information necessary to create the predicate for 
the new
-      //    filter
-      ListMultimap<RexInputRef,RexLiteral> columnConstantsMap = 
ArrayListMultimap.create();
       ImmutableList<RexNode> operands = RexUtil.flattenOr(((RexCall) 
condition).getOperands());
       if (operands.size() < minNumORClauses) {
         // We bail out
         return null;
       }
+      List<ConstraintGroup> allNodes = new ArrayList<>();
+      List<ConstraintGroup> processedNodes = new ArrayList<>();
       for (int i = 0; i < operands.size(); i++) {
-        final List<RexNode> conjunctions = 
RelOptUtil.conjunctions(operands.get(i));
-        for (RexNode conjunction: conjunctions) {
-          // 1.1. If it is not a RexCall, we bail out
-          if (!(conjunction instanceof RexCall)) {
-            return null;
-          }
-          // 1.2. We extract the information that we need
-          RexCall conjCall = (RexCall) conjunction;
-          if(conjCall.getOperator().getKind() == SqlKind.EQUALS) {
-            if (conjCall.operands.get(0) instanceof RexInputRef &&
-                    conjCall.operands.get(1) instanceof RexLiteral) {
-              RexInputRef ref = (RexInputRef) conjCall.operands.get(0);
-              RexLiteral literal = (RexLiteral) conjCall.operands.get(1);
-              columnConstantsMap.put(ref, literal);
-              if (columnConstantsMap.get(ref).size() != i+1) {
-                // If we have not added to this column before, we bail out
-                return null;
-              }
-            } else if (conjCall.operands.get(1) instanceof RexInputRef &&
-                    conjCall.operands.get(0) instanceof RexLiteral) {
-              RexInputRef ref = (RexInputRef) conjCall.operands.get(1);
-              RexLiteral literal = (RexLiteral) conjCall.operands.get(0);
-              columnConstantsMap.put(ref, literal);
-              if (columnConstantsMap.get(ref).size() != i+1) {
-                // If we have not added to this column before, we bail out
-                return null;
-              }
-            } else {
-              // Bail out
-              return null;
-            }
-          } else {
-            return null;
-          }
-        }
+        ConstraintGroup m = new ConstraintGroup(operands.get(i));
+        allNodes.add(m);
       }
 
-      // 3. We build the new predicate and return it
-      List<RexNode> newOperands = new ArrayList<RexNode>(operands.size());
-      // 3.1 Create structs
-      List<RexInputRef> columns = new ArrayList<RexInputRef>();
-      List<String> names = new ArrayList<String>();
-      ImmutableList.Builder<RelDataType> paramsTypes = ImmutableList.builder();
-      List<TypeInfo> structReturnType = new ArrayList<TypeInfo>();
-      ImmutableList.Builder<RelDataType> newOperandsTypes = 
ImmutableList.builder();
-      for (int i = 0; i < operands.size(); i++) {
-        List<RexLiteral> constantFields = new 
ArrayList<RexLiteral>(operands.size());
+      Multimap<Set<RexInputRef>, ConstraintGroup> assignmentGroups =
+          Multimaps.index(allNodes, ConstraintGroup.KEY_FUNCTION);
 
-        for (RexInputRef ref : columnConstantsMap.keySet()) {
-          // If any of the elements was not referenced by every operand, we 
bail out
-          if (columnConstantsMap.get(ref).size() <= i) {
-            return null;
-          }
-          RexLiteral columnConstant = columnConstantsMap.get(ref).get(i);
-          if (i == 0) {
-            columns.add(ref);
-            names.add(inputSchema.getFieldNames().get(ref.getIndex()));
-            paramsTypes.add(ref.getType());
-            structReturnType.add(TypeConverter.convert(ref.getType()));
-          }
-          constantFields.add(columnConstant);
+      for (Entry<Set<RexInputRef>, Collection<ConstraintGroup>> sa : 
assignmentGroups.asMap().entrySet()) {
+        // skip opaque
+        if (sa.getKey().size() == 0) {
+          continue;
         }
-
-        if (i == 0) {
-          RexNode columnsRefs;
-          if (columns.size() == 1) {
-            columnsRefs = columns.get(0);
-          } else {
-            // Create STRUCT clause
-            columnsRefs = rexBuilder.makeCall(SqlStdOperatorTable.ROW, 
columns);
-          }
-          newOperands.add(columnsRefs);
-          newOperandsTypes.add(columnsRefs.getType());
+        // not enough equalities should not be handled
+        if (sa.getValue().size() < 2 || sa.getValue().size() < 
minNumORClauses) {
+          continue;
         }
-        RexNode values;
-        if (constantFields.size() == 1) {
-          values = constantFields.get(0);
-        } else {
-          // Create STRUCT clause
-          values = rexBuilder.makeCall(SqlStdOperatorTable.ROW, 
constantFields);
-        }
-        newOperands.add(values);
-        newOperandsTypes.add(values.getType());
+
+        allNodes.add(new ConstraintGroup(buildInFor(sa.getKey(), 
sa.getValue())));
+        processedNodes.addAll(sa.getValue());
+      }
+
+      if (processedNodes.isEmpty()) {
+        return null;
       }
+      allNodes.removeAll(processedNodes);
+      List<RexNode> ops = new ArrayList<>();
+      for (ConstraintGroup mx : allNodes) {
+        ops.add(mx.originalRexNode);
+      }
+      if (ops.size() == 1) {
+        return ops.get(0);
+      } else {
+        return rexBuilder.makeCall(SqlStdOperatorTable.OR, ops);
+      }
+
+    }
 
-      // 4. Create and return IN clause
-      return rexBuilder.makeCall(HiveIn.INSTANCE, newOperands);
+    private RexNode buildInFor(Set<RexInputRef> set, 
Collection<ConstraintGroup> value) throws SemanticException {
+
+      List<RexInputRef> columns = new ArrayList<RexInputRef>();
+      columns.addAll(set);
+      List<RexNode >operands = new ArrayList<>();
+
+      operands.add(useStructIfNeeded(columns));
+      for (ConstraintGroup node : value) {
+        List<RexNode> values = node.getValuesInOrder(columns);
+        operands.add(useStructIfNeeded(values));
+      }
+
+      return rexBuilder.makeCall(HiveIn.INSTANCE, operands);
+    }
+
+    private RexNode useStructIfNeeded(List<? extends RexNode> columns) {
+      // Create STRUCT clause
+      if (columns.size() == 1) {
+        return columns.get(0);
+      } else {
+        return rexBuilder.makeCall(SqlStdOperatorTable.ROW, columns);
+      }
     }
 
   }
@@ -337,7 +418,7 @@ public abstract class HivePointLookupOptimizerRule extends 
RelOptRule {
       switch (call.getKind()) {
         case AND:
           // IN clauses need to be combined by keeping only common elements
-          operands = Lists.newArrayList(RexUtil.flattenAnd(((RexCall) 
call).getOperands()));
+          operands = 
Lists.newArrayList(RexUtil.flattenAnd(call.getOperands()));
           for (int i = 0; i < operands.size(); i++) {
             RexNode operand = operands.get(i);
             if (operand.getKind() == SqlKind.IN) {
@@ -374,7 +455,7 @@ public abstract class HivePointLookupOptimizerRule extends 
RelOptRule {
           break;
         case OR:
           // IN clauses need to be combined by keeping all elements
-          operands = Lists.newArrayList(RexUtil.flattenOr(((RexCall) 
call).getOperands()));
+          operands = Lists.newArrayList(RexUtil.flattenOr(call.getOperands()));
           for (int i = 0; i < operands.size(); i++) {
             RexNode operand = operands.get(i);
             if (operand.getKind() == SqlKind.IN) {

http://git-wip-us.apache.org/repos/asf/hive/blob/93de9504/ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/rules/TestHivePointLookupOptimizerRule.java
----------------------------------------------------------------------
diff --git 
a/ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/rules/TestHivePointLookupOptimizerRule.java
 
b/ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/rules/TestHivePointLookupOptimizerRule.java
new file mode 100644
index 0000000..a593201
--- /dev/null
+++ 
b/ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/rules/TestHivePointLookupOptimizerRule.java
@@ -0,0 +1,174 @@
+/*
+ * 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;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptSchema;
+import org.apache.calcite.plan.hep.HepPlanner;
+import org.apache.calcite.plan.hep.HepProgramBuilder;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories;
+import org.apache.hadoop.hive.ql.optimizer.calcite.RelOptHiveTable;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFilter;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Matchers;
+import org.mockito.Mock;
+import org.mockito.Mockito;
+import org.mockito.runners.MockitoJUnitRunner;
+
+@RunWith(MockitoJUnitRunner.class)
+public class TestHivePointLookupOptimizerRule {
+
+  @Mock
+  private RelOptSchema schemaMock;
+  @Mock
+  RelOptHiveTable tableMock;
+  @Mock
+  Table hiveTableMDMock;
+
+  private HepPlanner planner;
+  private RelBuilder builder;
+
+  @SuppressWarnings("unused")
+  private static class MyRecord {
+    public int f1;
+    public int f2;
+  }
+
+  @Before
+  public void before() {
+    HepProgramBuilder programBuilder = new HepProgramBuilder();
+    programBuilder.addRuleInstance(new 
HivePointLookupOptimizerRule.FilterCondition(2));
+
+    planner = new HepPlanner(programBuilder.build());
+
+    JavaTypeFactoryImpl typeFactory = new JavaTypeFactoryImpl();
+    RexBuilder rexBuilder = new RexBuilder(typeFactory);
+    final RelOptCluster optCluster = RelOptCluster.create(planner, rexBuilder);
+    RelDataType rowTypeMock = typeFactory.createStructType(MyRecord.class);
+    Mockito.doReturn(rowTypeMock).when(tableMock).getRowType();
+    
Mockito.doReturn(tableMock).when(schemaMock).getTableForMember(Matchers.any());
+    Mockito.doReturn(hiveTableMDMock).when(tableMock).getHiveTableMD();
+
+    builder = HiveRelFactories.HIVE_BUILDER.create(optCluster, schemaMock);
+
+  }
+
+  public RexNode or(RexNode... args) {
+    return builder.call(SqlStdOperatorTable.OR, args);
+  }
+
+  public RexNode and(RexNode... args) {
+    return builder.call(SqlStdOperatorTable.AND, args);
+  }
+
+  public RexNode eq(String field, int value) {
+    return builder.call(SqlStdOperatorTable.EQUALS,
+        builder.field(field), builder.literal(value));
+  }
+
+  @Test
+  public void testSimpleCase() {
+
+    // @formatter:off
+    final RelNode basePlan = builder
+          .scan("t")
+          .filter(
+              and(
+                or(
+                    eq("f1",1),
+                    eq("f1",2)
+                    ),
+                or(
+                    eq("f2",3),
+                    eq("f2",4)
+                    )
+                )
+              )
+          .build();
+    // @formatter:on
+
+    planner.setRoot(basePlan);
+    RelNode optimizedRelNode = planner.findBestExp();
+
+    HiveFilter filter = (HiveFilter) optimizedRelNode;
+    RexNode condition = filter.getCondition();
+    assertEquals("AND(IN($0, 1, 2), IN($1, 3, 4))", condition.toString());
+  }
+
+  @Test
+  public void testSimpleStructCase() {
+
+    // @formatter:off
+    final RelNode basePlan = builder
+          .scan("t")
+          .filter(
+              or(
+                  and( eq("f1",1),eq("f2",1)),
+                  and( eq("f1",2),eq("f2",2))
+                  )
+              )
+          .build();
+    // @formatter:on
+
+    planner.setRoot(basePlan);
+    RelNode optimizedRelNode = planner.findBestExp();
+
+    HiveFilter filter = (HiveFilter) optimizedRelNode;
+    RexNode condition = filter.getCondition();
+    assertEquals("IN(ROW($0, $1), ROW(1, 1), ROW(2, 2))", 
condition.toString());
+  }
+
+  /** Despite the fact that f2=99 is there...the extraction should happen */
+  @Test
+  public void testObscuredSimple() {
+
+    // @formatter:off
+    final RelNode basePlan = builder
+          .scan("t")
+          .filter(
+              or(
+                  eq("f2",99),
+                  eq("f1",1),
+                  eq("f1",2)
+                  )
+              )
+          .build();
+    // @formatter:on
+
+    planner.setRoot(basePlan);
+    RelNode optimizedRelNode = planner.findBestExp();
+
+    HiveFilter filter = (HiveFilter) optimizedRelNode;
+    RexNode condition = filter.getCondition();
+    System.out.println(condition);
+    assertEquals("OR(IN($0, 1, 2), =($1, 99))", condition.toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/93de9504/ql/src/test/results/clientpositive/druid/druidmini_test_ts.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/druid/druidmini_test_ts.q.out 
b/ql/src/test/results/clientpositive/druid/druidmini_test_ts.q.out
index 9c624c2..f077fb2 100644
--- a/ql/src/test/results/clientpositive/druid/druidmini_test_ts.q.out
+++ b/ql/src/test/results/clientpositive/druid/druidmini_test_ts.q.out
@@ -507,7 +507,7 @@ STAGE PLANS:
           properties:
             druid.fieldNames vc,cstring2
             druid.fieldTypes timestamp with local time zone,string
-            druid.query.json 
{"queryType":"scan","dataSource":"default.druid_table_alltypesorc","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"],"filter":{"type":"or","fields":[{"type":"selector","dimension":"cstring2","value":"user1"},{"type":"selector","dimension":"__time","value":"2010-01-01T08:00:00.000Z","extractionFn":{"type":"timeFormat","format":"yyyy-MM-dd'T'HH:mm:ss.SSS'Z'","timeZone":"UTC"}},{"type":"selector","dimension":"__time","value":"2011-01-01T08:00:00.000Z","extractionFn":{"type":"timeFormat","format":"yyyy-MM-dd'T'HH:mm:ss.SSS'Z'","timeZone":"UTC"}}]},"virtualColumns":[{"type":"expression","name":"vc","expression":"\"__time\"","outputType":"LONG"}],"columns":["vc","cstring2"],"resultFormat":"compactedList"}
+            druid.query.json 
{"queryType":"scan","dataSource":"default.druid_table_alltypesorc","intervals":["1900-01-01T00:00:00.000Z/3000-01-01T00:00:00.000Z"],"filter":{"type":"or","fields":[{"type":"in","dimension":"__time","values":["2010-01-01T08:00:00.000Z","2011-01-01T08:00:00.000Z"],"extractionFn":{"type":"timeFormat","format":"yyyy-MM-dd'T'HH:mm:ss.SSS'Z'","timeZone":"UTC"}},{"type":"selector","dimension":"cstring2","value":"user1"}]},"virtualColumns":[{"type":"expression","name":"vc","expression":"\"__time\"","outputType":"LONG"}],"columns":["vc","cstring2"],"resultFormat":"compactedList"}
             druid.query.type scan
           Select Operator
             expressions: vc (type: timestamp with local time zone), cstring2 
(type: string)

http://git-wip-us.apache.org/repos/asf/hive/blob/93de9504/ql/src/test/results/clientpositive/llap/bucketpruning1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/bucketpruning1.q.out 
b/ql/src/test/results/clientpositive/llap/bucketpruning1.q.out
index 260ba1c..55442ad 100644
--- a/ql/src/test/results/clientpositive/llap/bucketpruning1.q.out
+++ b/ql/src/test/results/clientpositive/llap/bucketpruning1.q.out
@@ -1542,9 +1542,6 @@ PREHOOK: type: QUERY
 POSTHOOK: query: explain extended
 select * from srcbucket_pruned where key = 1 or value = "One" or key = 2
 POSTHOOK: type: QUERY
-OPTIMIZED SQL: SELECT `key`, `value`, `ds`
-FROM `default`.`srcbucket_pruned`
-WHERE `key` = 1 OR `value` = 'One' OR `key` = 2
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
   Stage-0 depends on stages: Stage-1
@@ -1558,12 +1555,12 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: srcbucket_pruned
-                  filterExpr: ((key = 1) or (value = 'One') or (key = 2)) 
(type: boolean)
+                  filterExpr: ((key) IN (1, 2) or (value = 'One')) (type: 
boolean)
                   Statistics: Num rows: 1 Data size: 372 Basic stats: COMPLETE 
Column stats: PARTIAL
                   GatherStats: false
                   Filter Operator
                     isSamplingPred: false
-                    predicate: ((key = 1) or (key = 2) or (value = 'One')) 
(type: boolean)
+                    predicate: ((key) IN (1, 2) or (value = 'One')) (type: 
boolean)
                     Statistics: Num rows: 1 Data size: 372 Basic stats: 
COMPLETE Column stats: PARTIAL
                     Select Operator
                       expressions: key (type: int), value (type: string), ds 
(type: string)

http://git-wip-us.apache.org/repos/asf/hive/blob/93de9504/ql/src/test/results/clientpositive/perf/spark/query15.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/spark/query15.q.out 
b/ql/src/test/results/clientpositive/perf/spark/query15.q.out
index 67684f6..3d6fbda 100644
--- a/ql/src/test/results/clientpositive/perf/spark/query15.q.out
+++ b/ql/src/test/results/clientpositive/perf/spark/query15.q.out
@@ -157,7 +157,7 @@ STAGE PLANS:
                 outputColumnNames: _col3, _col4, _col7
                 Statistics: Num rows: 348467716 Data size: 47189528877 Basic 
stats: COMPLETE Column stats: NONE
                 Filter Operator
-                  predicate: ((_col3 = 'CA') or (_col3 = 'GA') or (_col3 = 
'WA') or (_col7 > 500) or (substr(_col4, 1, 5)) IN ('85669', '86197', '88274', 
'83405', '86475', '85392', '85460', '80348', '81792')) (type: boolean)
+                  predicate: ((_col3) IN ('CA', 'WA', 'GA') or (_col7 > 500) 
or (substr(_col4, 1, 5)) IN ('85669', '86197', '88274', '83405', '86475', 
'85392', '85460', '80348', '81792')) (type: boolean)
                   Statistics: Num rows: 348467716 Data size: 47189528877 Basic 
stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: _col4 (type: string), _col7 (type: 
decimal(7,2))

http://git-wip-us.apache.org/repos/asf/hive/blob/93de9504/ql/src/test/results/clientpositive/perf/spark/query47.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/spark/query47.q.out 
b/ql/src/test/results/clientpositive/perf/spark/query47.q.out
index a9b5092..4a66d0b 100644
--- a/ql/src/test/results/clientpositive/perf/spark/query47.q.out
+++ b/ql/src/test/results/clientpositive/perf/spark/query47.q.out
@@ -245,20 +245,20 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: date_dim
-                  filterExpr: (((d_year = 2000) or ((d_year = 1999) and (d_moy 
= 12)) or ((d_year = 2001) and (d_moy = 1))) and d_date_sk is not null) (type: 
boolean)
+                  filterExpr: (((struct(d_year,d_moy)) IN (const 
struct(1999,12), const struct(2001,1)) or (d_year = 2000)) and d_date_sk is not 
null) (type: boolean)
                   Statistics: Num rows: 73049 Data size: 81741831 Basic stats: 
COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: (((d_year = 2000) or ((d_year = 1999) and 
(d_moy = 12)) or ((d_year = 2001) and (d_moy = 1))) and d_date_sk is not null) 
(type: boolean)
-                    Statistics: Num rows: 73048 Data size: 81740712 Basic 
stats: COMPLETE Column stats: NONE
+                    predicate: (((struct(d_year,d_moy)) IN (const 
struct(1999,12), const struct(2001,1)) or (d_year = 2000)) and d_date_sk is not 
null) (type: boolean)
+                    Statistics: Num rows: 73049 Data size: 81741831 Basic 
stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: d_date_sk (type: int), d_year (type: int), 
d_moy (type: int)
                       outputColumnNames: _col0, _col1, _col2
-                      Statistics: Num rows: 73048 Data size: 81740712 Basic 
stats: COMPLETE Column stats: NONE
+                      Statistics: Num rows: 73049 Data size: 81741831 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: 73048 Data size: 81740712 Basic 
stats: COMPLETE Column stats: NONE
+                        Statistics: Num rows: 73049 Data size: 81741831 Basic 
stats: COMPLETE Column stats: NONE
                         value expressions: _col1 (type: int), _col2 (type: int)
             Execution mode: vectorized
         Map 18 
@@ -305,20 +305,20 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: date_dim
-                  filterExpr: (((d_year = 2000) or ((d_year = 1999) and (d_moy 
= 12)) or ((d_year = 2001) and (d_moy = 1))) and d_date_sk is not null) (type: 
boolean)
+                  filterExpr: (((struct(d_year,d_moy)) IN (const 
struct(1999,12), const struct(2001,1)) or (d_year = 2000)) and d_date_sk is not 
null) (type: boolean)
                   Statistics: Num rows: 73049 Data size: 81741831 Basic stats: 
COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: (((d_year = 2000) or ((d_year = 1999) and 
(d_moy = 12)) or ((d_year = 2001) and (d_moy = 1))) and d_date_sk is not null) 
(type: boolean)
-                    Statistics: Num rows: 73048 Data size: 81740712 Basic 
stats: COMPLETE Column stats: NONE
+                    predicate: (((struct(d_year,d_moy)) IN (const 
struct(1999,12), const struct(2001,1)) or (d_year = 2000)) and d_date_sk is not 
null) (type: boolean)
+                    Statistics: Num rows: 73049 Data size: 81741831 Basic 
stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: d_date_sk (type: int), d_year (type: int), 
d_moy (type: int)
                       outputColumnNames: _col0, _col1, _col2
-                      Statistics: Num rows: 73048 Data size: 81740712 Basic 
stats: COMPLETE Column stats: NONE
+                      Statistics: Num rows: 73049 Data size: 81741831 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: 73048 Data size: 81740712 Basic 
stats: COMPLETE Column stats: NONE
+                        Statistics: Num rows: 73049 Data size: 81741831 Basic 
stats: COMPLETE Column stats: NONE
                         value expressions: _col1 (type: int), _col2 (type: int)
             Execution mode: vectorized
         Map 26 
@@ -345,20 +345,20 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: date_dim
-                  filterExpr: (((d_year = 2000) or ((d_year = 1999) and (d_moy 
= 12)) or ((d_year = 2001) and (d_moy = 1))) and d_date_sk is not null) (type: 
boolean)
+                  filterExpr: (((struct(d_year,d_moy)) IN (const 
struct(1999,12), const struct(2001,1)) or (d_year = 2000)) and d_date_sk is not 
null) (type: boolean)
                   Statistics: Num rows: 73049 Data size: 81741831 Basic stats: 
COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: (((d_year = 2000) or ((d_year = 1999) and 
(d_moy = 12)) or ((d_year = 2001) and (d_moy = 1))) and d_date_sk is not null) 
(type: boolean)
-                    Statistics: Num rows: 73048 Data size: 81740712 Basic 
stats: COMPLETE Column stats: NONE
+                    predicate: (((struct(d_year,d_moy)) IN (const 
struct(1999,12), const struct(2001,1)) or (d_year = 2000)) and d_date_sk is not 
null) (type: boolean)
+                    Statistics: Num rows: 73049 Data size: 81741831 Basic 
stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: d_date_sk (type: int), d_year (type: int), 
d_moy (type: int)
                       outputColumnNames: _col0, _col1, _col2
-                      Statistics: Num rows: 73048 Data size: 81740712 Basic 
stats: COMPLETE Column stats: NONE
+                      Statistics: Num rows: 73049 Data size: 81741831 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: 73048 Data size: 81740712 Basic 
stats: COMPLETE Column stats: NONE
+                        Statistics: Num rows: 73049 Data size: 81741831 Basic 
stats: COMPLETE Column stats: NONE
                         value expressions: _col1 (type: int), _col2 (type: int)
             Execution mode: vectorized
         Map 9 

http://git-wip-us.apache.org/repos/asf/hive/blob/93de9504/ql/src/test/results/clientpositive/perf/spark/query57.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/spark/query57.q.out 
b/ql/src/test/results/clientpositive/perf/spark/query57.q.out
index 6785ee9..502d5f7 100644
--- a/ql/src/test/results/clientpositive/perf/spark/query57.q.out
+++ b/ql/src/test/results/clientpositive/perf/spark/query57.q.out
@@ -259,20 +259,20 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: date_dim
-                  filterExpr: (((d_year = 2000) or ((d_year = 1999) and (d_moy 
= 12)) or ((d_year = 2001) and (d_moy = 1))) and d_date_sk is not null) (type: 
boolean)
+                  filterExpr: (((struct(d_year,d_moy)) IN (const 
struct(1999,12), const struct(2001,1)) or (d_year = 2000)) and d_date_sk is not 
null) (type: boolean)
                   Statistics: Num rows: 73049 Data size: 81741831 Basic stats: 
COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: (((d_year = 2000) or ((d_year = 1999) and 
(d_moy = 12)) or ((d_year = 2001) and (d_moy = 1))) and d_date_sk is not null) 
(type: boolean)
-                    Statistics: Num rows: 73048 Data size: 81740712 Basic 
stats: COMPLETE Column stats: NONE
+                    predicate: (((struct(d_year,d_moy)) IN (const 
struct(1999,12), const struct(2001,1)) or (d_year = 2000)) and d_date_sk is not 
null) (type: boolean)
+                    Statistics: Num rows: 73049 Data size: 81741831 Basic 
stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: d_date_sk (type: int), d_year (type: int), 
d_moy (type: int)
                       outputColumnNames: _col0, _col1, _col2
-                      Statistics: Num rows: 73048 Data size: 81740712 Basic 
stats: COMPLETE Column stats: NONE
+                      Statistics: Num rows: 73049 Data size: 81741831 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: 73048 Data size: 81740712 Basic 
stats: COMPLETE Column stats: NONE
+                        Statistics: Num rows: 73049 Data size: 81741831 Basic 
stats: COMPLETE Column stats: NONE
                         value expressions: _col1 (type: int), _col2 (type: int)
             Execution mode: vectorized
         Map 19 
@@ -319,20 +319,20 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: date_dim
-                  filterExpr: (((d_year = 2000) or ((d_year = 1999) and (d_moy 
= 12)) or ((d_year = 2001) and (d_moy = 1))) and d_date_sk is not null) (type: 
boolean)
+                  filterExpr: (((struct(d_year,d_moy)) IN (const 
struct(1999,12), const struct(2001,1)) or (d_year = 2000)) and d_date_sk is not 
null) (type: boolean)
                   Statistics: Num rows: 73049 Data size: 81741831 Basic stats: 
COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: (((d_year = 2000) or ((d_year = 1999) and 
(d_moy = 12)) or ((d_year = 2001) and (d_moy = 1))) and d_date_sk is not null) 
(type: boolean)
-                    Statistics: Num rows: 73048 Data size: 81740712 Basic 
stats: COMPLETE Column stats: NONE
+                    predicate: (((struct(d_year,d_moy)) IN (const 
struct(1999,12), const struct(2001,1)) or (d_year = 2000)) and d_date_sk is not 
null) (type: boolean)
+                    Statistics: Num rows: 73049 Data size: 81741831 Basic 
stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: d_date_sk (type: int), d_year (type: int), 
d_moy (type: int)
                       outputColumnNames: _col0, _col1, _col2
-                      Statistics: Num rows: 73048 Data size: 81740712 Basic 
stats: COMPLETE Column stats: NONE
+                      Statistics: Num rows: 73049 Data size: 81741831 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: 73048 Data size: 81740712 Basic 
stats: COMPLETE Column stats: NONE
+                        Statistics: Num rows: 73049 Data size: 81741831 Basic 
stats: COMPLETE Column stats: NONE
                         value expressions: _col1 (type: int), _col2 (type: int)
             Execution mode: vectorized
         Map 27 
@@ -359,20 +359,20 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: date_dim
-                  filterExpr: (((d_year = 2000) or ((d_year = 1999) and (d_moy 
= 12)) or ((d_year = 2001) and (d_moy = 1))) and d_date_sk is not null) (type: 
boolean)
+                  filterExpr: (((struct(d_year,d_moy)) IN (const 
struct(1999,12), const struct(2001,1)) or (d_year = 2000)) and d_date_sk is not 
null) (type: boolean)
                   Statistics: Num rows: 73049 Data size: 81741831 Basic stats: 
COMPLETE Column stats: NONE
                   Filter Operator
-                    predicate: (((d_year = 2000) or ((d_year = 1999) and 
(d_moy = 12)) or ((d_year = 2001) and (d_moy = 1))) and d_date_sk is not null) 
(type: boolean)
-                    Statistics: Num rows: 73048 Data size: 81740712 Basic 
stats: COMPLETE Column stats: NONE
+                    predicate: (((struct(d_year,d_moy)) IN (const 
struct(1999,12), const struct(2001,1)) or (d_year = 2000)) and d_date_sk is not 
null) (type: boolean)
+                    Statistics: Num rows: 73049 Data size: 81741831 Basic 
stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: d_date_sk (type: int), d_year (type: int), 
d_moy (type: int)
                       outputColumnNames: _col0, _col1, _col2
-                      Statistics: Num rows: 73048 Data size: 81740712 Basic 
stats: COMPLETE Column stats: NONE
+                      Statistics: Num rows: 73049 Data size: 81741831 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: 73048 Data size: 81740712 Basic 
stats: COMPLETE Column stats: NONE
+                        Statistics: Num rows: 73049 Data size: 81741831 Basic 
stats: COMPLETE Column stats: NONE
                         value expressions: _col1 (type: int), _col2 (type: int)
             Execution mode: vectorized
         Reducer 12 

http://git-wip-us.apache.org/repos/asf/hive/blob/93de9504/ql/src/test/results/clientpositive/perf/tez/query15.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/tez/query15.q.out 
b/ql/src/test/results/clientpositive/perf/tez/query15.q.out
index e1eca99..3c7ae66 100644
--- a/ql/src/test/results/clientpositive/perf/tez/query15.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/query15.q.out
@@ -71,7 +71,7 @@ Stage-0
                       Select Operator [SEL_23] (rows=348467716 width=135)
                         Output:["_col4","_col7"]
                         Filter Operator [FIL_22] (rows=348467716 width=135)
-                          predicate:((_col3 = 'CA') or (_col3 = 'GA') or 
(_col3 = 'WA') or (_col7 > 500) or (substr(_col4, 1, 5)) IN ('85669', '86197', 
'88274', '83405', '86475', '85392', '85460', '80348', '81792'))
+                          predicate:((_col3) IN ('CA', 'WA', 'GA') or (_col7 > 
500) or (substr(_col4, 1, 5)) IN ('85669', '86197', '88274', '83405', '86475', 
'85392', '85460', '80348', '81792'))
                           Merge Join Operator [MERGEJOIN_77] (rows=348467716 
width=135)
                             
Conds:RS_19._col0=RS_20._col1(Inner),Output:["_col3","_col4","_col7"]
                           <-Reducer 2 [SIMPLE_EDGE]

http://git-wip-us.apache.org/repos/asf/hive/blob/93de9504/ql/src/test/results/clientpositive/perf/tez/query47.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/tez/query47.q.out 
b/ql/src/test/results/clientpositive/perf/tez/query47.q.out
index bd17808..f931483 100644
--- a/ql/src/test/results/clientpositive/perf/tez/query47.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/query47.q.out
@@ -199,10 +199,10 @@ Stage-0
                                                         <-Map 12 [SIMPLE_EDGE] 
vectorized
                                                           SHUFFLE [RS_282]
                                                             PartitionCols:_col0
-                                                            Select Operator 
[SEL_281] (rows=73048 width=1119)
+                                                            Select Operator 
[SEL_281] (rows=73049 width=1119)
                                                               
Output:["_col0","_col1","_col2"]
-                                                              Filter Operator 
[FIL_280] (rows=73048 width=1119)
-                                                                
predicate:(((d_year = 2000) or ((d_year = 1999) and (d_moy = 12)) or ((d_year = 
2001) and (d_moy = 1))) and d_date_sk is not null)
+                                                              Filter Operator 
[FIL_280] (rows=73049 width=1119)
+                                                                
predicate:(((struct(d_year,d_moy)) IN (const struct(1999,12), const 
struct(2001,1)) or (d_year = 2000)) and d_date_sk is not null)
                                                                 TableScan 
[TS_73] (rows=73049 width=1119)
                                                                   
default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year","d_moy"]
                                                         <-Map 1 [SIMPLE_EDGE] 
vectorized
@@ -222,7 +222,7 @@ Stage-0
                                                                       SHUFFLE 
[RS_285]
                                                                         Group 
By Operator [GBY_284] (rows=1 width=12)
                                                                           
Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0,
 expectedEntries=1000000)"]
-                                                                          
Select Operator [SEL_283] (rows=73048 width=1119)
+                                                                          
Select Operator [SEL_283] (rows=73049 width=1119)
                                                                             
Output:["_col0"]
                                                                              
Please refer to the previous Select Operator [SEL_281]
                                                                 <-Reducer 15 
[BROADCAST_EDGE] vectorized

http://git-wip-us.apache.org/repos/asf/hive/blob/93de9504/ql/src/test/results/clientpositive/perf/tez/query57.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/tez/query57.q.out 
b/ql/src/test/results/clientpositive/perf/tez/query57.q.out
index 1d3c17d..fed340a 100644
--- a/ql/src/test/results/clientpositive/perf/tez/query57.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/query57.q.out
@@ -193,10 +193,10 @@ Stage-0
                                                         <-Map 12 [SIMPLE_EDGE] 
vectorized
                                                           SHUFFLE [RS_282]
                                                             PartitionCols:_col0
-                                                            Select Operator 
[SEL_281] (rows=73048 width=1119)
+                                                            Select Operator 
[SEL_281] (rows=73049 width=1119)
                                                               
Output:["_col0","_col1","_col2"]
-                                                              Filter Operator 
[FIL_280] (rows=73048 width=1119)
-                                                                
predicate:(((d_year = 2000) or ((d_year = 1999) and (d_moy = 12)) or ((d_year = 
2001) and (d_moy = 1))) and d_date_sk is not null)
+                                                              Filter Operator 
[FIL_280] (rows=73049 width=1119)
+                                                                
predicate:(((struct(d_year,d_moy)) IN (const struct(1999,12), const 
struct(2001,1)) or (d_year = 2000)) and d_date_sk is not null)
                                                                 TableScan 
[TS_73] (rows=73049 width=1119)
                                                                   
default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year","d_moy"]
                                                         <-Map 1 [SIMPLE_EDGE] 
vectorized
@@ -216,7 +216,7 @@ Stage-0
                                                                       SHUFFLE 
[RS_285]
                                                                         Group 
By Operator [GBY_284] (rows=1 width=12)
                                                                           
Output:["_col0","_col1","_col2"],aggregations:["min(_col0)","max(_col0)","bloom_filter(_col0,
 expectedEntries=1000000)"]
-                                                                          
Select Operator [SEL_283] (rows=73048 width=1119)
+                                                                          
Select Operator [SEL_283] (rows=73049 width=1119)
                                                                             
Output:["_col0"]
                                                                              
Please refer to the previous Select Operator [SEL_281]
                                                                 <-Reducer 15 
[BROADCAST_EDGE] vectorized

Reply via email to