You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by xu...@apache.org on 2015/09/09 09:08:35 UTC

[05/50] [abbrv] hive git commit: HIVE-11573: PointLookupOptimizer can be pessimistic at a low nDV (Gopal V, reviewed by Jesus Camacho Rodriguez)

HIVE-11573: PointLookupOptimizer can be pessimistic at a low nDV (Gopal V, reviewed by Jesus Camacho Rodriguez)


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

Branch: refs/heads/beeline-cli
Commit: b247cac4fc3814e422d4f5d5aad96a1c6e385a7b
Parents: 037fb02
Author: Gopal V <go...@apache.org>
Authored: Thu Aug 27 09:50:08 2015 +0200
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Thu Aug 27 09:50:08 2015 +0200

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |    4 +
 .../hadoop/hive/ql/optimizer/Optimizer.java     |   17 +-
 .../hive/ql/optimizer/PointLookupOptimizer.java |  102 +-
 .../queries/clientpositive/flatten_and_or.q     |    4 +-
 .../test/queries/clientpositive/pointlookup.q   |   59 +
 .../test/queries/clientpositive/pointlookup2.q  |   51 +
 .../alter_partition_coltype.q.out               |   12 +-
 .../clientpositive/annotate_stats_filter.q.out  |    8 +-
 .../results/clientpositive/flatten_and_or.q.out |    8 +-
 ql/src/test/results/clientpositive/pcr.q.out    |   12 +-
 .../results/clientpositive/pointlookup.q.out    |  198 +++
 .../results/clientpositive/pointlookup2.q.out   | 1647 ++++++++++++++++++
 .../results/clientpositive/ppd_transform.q.out  |   12 +-
 .../test/results/clientpositive/spark/pcr.q.out |   12 +-
 .../clientpositive/spark/ppd_transform.q.out    |   12 +-
 .../clientpositive/spark/vectorized_case.q.out  |    2 +-
 .../clientpositive/tez/explainuser_1.q.out      |    2 +-
 .../clientpositive/tez/vectorized_case.q.out    |    2 +-
 .../clientpositive/vectorized_case.q.out        |    9 +-
 19 files changed, 2118 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/b247cac4/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 8706a2d..8a00079 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -1190,6 +1190,10 @@ public class HiveConf extends Configuration {
         "Whether to push predicates down into storage handlers.  Ignored when hive.optimize.ppd is false."),
     HIVEPOINTLOOKUPOPTIMIZER("hive.optimize.point.lookup", true,
          "Whether to transform OR clauses in Filter operators into IN clauses"),
+    HIVEPOINTLOOKUPOPTIMIZERMIN("hive.optimize.point.lookup.min", 31,
+             "Minimum number of OR clauses needed to transform into IN clauses"),
+    HIVEPOINTLOOKUPOPTIMIZEREXTRACT("hive.optimize.point.lookup.extract", true,
+                 "Extract partial expressions when optimizing point lookup IN clauses"),
     // Constant propagation optimizer
     HIVEOPTCONSTANTPROPAGATION("hive.optimize.constant.propagation", true, "Whether to enable constant propagation optimizer"),
     HIVEIDENTITYPROJECTREMOVER("hive.optimize.remove.identity.project", true, "Removes identity project from operator tree"),

http://git-wip-us.apache.org/repos/asf/hive/blob/b247cac4/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java
index 14f362f..439f616 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java
@@ -68,6 +68,18 @@ public class Optimizer {
 
     // Add the transformation that computes the lineage information.
     transformations.add(new Generator());
+
+    // Try to transform OR predicates in Filter into simpler IN clauses first
+    if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEPOINTLOOKUPOPTIMIZER)) {
+      final int min = HiveConf.getIntVar(hiveConf,
+          HiveConf.ConfVars.HIVEPOINTLOOKUPOPTIMIZERMIN);
+      final boolean extract = HiveConf.getBoolVar(hiveConf,
+          HiveConf.ConfVars.HIVEPOINTLOOKUPOPTIMIZEREXTRACT);
+      final boolean testMode = HiveConf.getBoolVar(hiveConf,
+          HiveConf.ConfVars.HIVE_IN_TEST);
+      transformations.add(new PointLookupOptimizer(min, extract, testMode));
+    }
+
     if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTPPD)) {
       transformations.add(new PredicateTransitivePropagate());
       if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTCONSTANTPROPAGATION)) {
@@ -82,11 +94,6 @@ public class Optimizer {
         transformations.add(new ConstantPropagate());
     }
 
-    // Try to transform OR predicates in Filter into IN clauses.
-    if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEPOINTLOOKUPOPTIMIZER)) {
-      transformations.add(new PointLookupOptimizer());
-    }
-
     if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTPPD)) {
       transformations.add(new PartitionPruner());
       transformations.add(new PartitionConditionRemover());

http://git-wip-us.apache.org/repos/asf/hive/blob/b247cac4/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PointLookupOptimizer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PointLookupOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PointLookupOptimizer.java
index 6a8acec..d83636d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PointLookupOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PointLookupOptimizer.java
@@ -18,10 +18,14 @@
 package org.apache.hadoop.hive.ql.optimizer;
 
 import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Comparator;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.Stack;
 
 import org.apache.calcite.util.Pair;
@@ -46,15 +50,18 @@ import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDesc.ExprNodeDescEqualityWrapper;
 import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFIn;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPAnd;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFStruct;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 
 import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.ImmutableSortedSet;
 import com.google.common.collect.ListMultimap;
 
 /**
@@ -71,7 +78,49 @@ public class PointLookupOptimizer implements Transform {
           GenericUDFIn.class.getAnnotation(Description.class).name();
   private static final String STRUCT_UDF =
           GenericUDFStruct.class.getAnnotation(Description.class).name();
+  private static final String AND_UDF =
+      GenericUDFOPAnd.class.getAnnotation(Description.class).name();
+
+  // these are closure-bound for all the walkers in context
+  public final int minOrExpr;
+  public final boolean extract;
+  public final boolean testMode;
+
+  /*
+   * Pass in configs and pre-create a parse context
+   */
+  public PointLookupOptimizer(final int min, final boolean extract, final boolean testMode) {
+    this.minOrExpr = min;
+    this.extract = extract;
+    this.testMode = testMode;
+  }
+
+  // Hash Set iteration isn't ordered, but force string sorted order
+  // to get a consistent test run.
+  private Collection<ExprNodeDescEqualityWrapper> sortForTests(
+      Set<ExprNodeDescEqualityWrapper> valuesExpr) {
+    if (!testMode) {
+      // normal case - sorting is wasted for an IN()
+      return valuesExpr;
+    }
+    final Collection<ExprNodeDescEqualityWrapper> sortedValues;
+
+    sortedValues = ImmutableSortedSet.copyOf(
+        new Comparator<ExprNodeDescEqualityWrapper>() {
+          @Override
+          public int compare(ExprNodeDescEqualityWrapper w1,
+              ExprNodeDescEqualityWrapper w2) {
+            // fail if you find nulls (this is a test-code section)
+            if (w1.equals(w2)) {
+              return 0;
+            }
+            return w1.getExprNodeDesc().getExprString()
+                .compareTo(w2.getExprNodeDesc().getExprString());
+          }
+        }, valuesExpr);
 
+    return sortedValues;
+  }
 
   @Override
   public ParseContext transform(ParseContext pctx) throws SemanticException {
@@ -103,7 +152,9 @@ public class PointLookupOptimizer implements Transform {
         if (LOG.isDebugEnabled()) {
           LOG.debug("Generated new predicate with IN clause: " + newPredicate);
         }
-        filterOp.getConf().setOrigPredicate(predicate);
+        if (!extract) {
+          filterOp.getConf().setOrigPredicate(predicate);
+        }
         filterOp.getConf().setPredicate(newPredicate);
       }
 
@@ -140,8 +191,11 @@ public class PointLookupOptimizer implements Transform {
         return null;
       }
 
-      // 2. It is an OR operator
+      // 2. It is an OR operator with enough children
       List<ExprNodeDesc> children = fd.getChildren();
+      if (children.size() < minOrExpr) {
+        return null;
+      }
       ListMultimap<String,Pair<ExprNodeColumnDesc, ExprNodeConstantDesc>> columnConstantsMap =
               ArrayListMultimap.create();
       boolean modeAnd = false;
@@ -272,6 +326,50 @@ public class PointLookupOptimizer implements Transform {
       newPredicate = new ExprNodeGenericFuncDesc(TypeInfoFactory.booleanTypeInfo,
               FunctionRegistry.getFunctionInfo(IN_UDF).getGenericUDF(), newChildren);
 
+      if (extract && columns.size() > 1) {
+        final List<ExprNodeDesc> subExpr = new ArrayList<ExprNodeDesc>(columns.size()+1);
+
+        // extract pre-conditions for the tuple expressions
+        // (a,b) IN ((1,2),(2,3)) ->
+        //          ((a) IN (1,2) and b in (2,3)) and (a,b) IN ((1,2),(2,3))
+
+        for (String keyString : columnConstantsMap.keySet()) {
+          final Set<ExprNodeDescEqualityWrapper> valuesExpr = 
+              new HashSet<ExprNodeDescEqualityWrapper>(children.size());
+          final List<Pair<ExprNodeColumnDesc, ExprNodeConstantDesc>> partial = 
+              columnConstantsMap.get(keyString);
+          for (int i = 0; i < children.size(); i++) {
+            Pair<ExprNodeColumnDesc, ExprNodeConstantDesc> columnConstant = partial
+                .get(i);
+            valuesExpr
+                .add(new ExprNodeDescEqualityWrapper(columnConstant.right));
+          }
+          ExprNodeColumnDesc lookupCol = partial.get(0).left;
+          // generate a partial IN clause, if the column is a partition column
+          if (lookupCol.getIsPartitionColOrVirtualCol()
+              || valuesExpr.size() < children.size()) {
+            // optimize only nDV reductions
+            final List<ExprNodeDesc> inExpr = new ArrayList<ExprNodeDesc>();
+            inExpr.add(lookupCol);
+            for (ExprNodeDescEqualityWrapper value : sortForTests(valuesExpr)) {
+              inExpr.add(value.getExprNodeDesc());
+            }
+            subExpr.add(new ExprNodeGenericFuncDesc(
+                TypeInfoFactory.booleanTypeInfo, FunctionRegistry
+                    .getFunctionInfo(IN_UDF).getGenericUDF(), inExpr));
+          }
+        }
+        // loop complete, inspect the sub expressions generated
+        if (subExpr.size() > 0) {
+          // add the newPredicate to the end & produce an AND clause
+          subExpr.add(newPredicate);
+          newPredicate = new ExprNodeGenericFuncDesc(
+              TypeInfoFactory.booleanTypeInfo, FunctionRegistry
+                  .getFunctionInfo(AND_UDF).getGenericUDF(), subExpr);
+        }
+        // else, newPredicate is unmodified
+      }
+
       return newPredicate;
     }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/b247cac4/ql/src/test/queries/clientpositive/flatten_and_or.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/flatten_and_or.q b/ql/src/test/queries/clientpositive/flatten_and_or.q
index 6d65225..6c6e0f9 100644
--- a/ql/src/test/queries/clientpositive/flatten_and_or.q
+++ b/ql/src/test/queries/clientpositive/flatten_and_or.q
@@ -1,3 +1,5 @@
+set hive.optimize.point.lookup=false;
+
 explain
 SELECT key
 FROM src
@@ -14,4 +16,4 @@ WHERE
    AND value = '1') OR (key = '9'
    AND value = '1') OR (key = '10'
    AND value = '3'))
-;
\ No newline at end of file
+;

http://git-wip-us.apache.org/repos/asf/hive/blob/b247cac4/ql/src/test/queries/clientpositive/pointlookup.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/pointlookup.q b/ql/src/test/queries/clientpositive/pointlookup.q
new file mode 100644
index 0000000..1aef2ef
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/pointlookup.q
@@ -0,0 +1,59 @@
+explain
+SELECT key
+FROM src
+WHERE
+   ((key = '0'
+   AND value = '8') OR (key = '1'
+   AND value = '5') OR (key = '2'
+   AND value = '6') OR (key = '3'
+   AND value = '8') OR (key = '4'
+   AND value = '1') OR (key = '5'
+   AND value = '6') OR (key = '6'
+   AND value = '1') OR (key = '7'
+   AND value = '1') OR (key = '8'
+   AND value = '1') OR (key = '9'
+   AND value = '1') OR (key = '10'
+   AND value = '3'))
+;
+
+
+set hive.optimize.point.lookup.min=3;
+set hive.optimize.point.lookup.extract=false;
+
+explain
+SELECT key
+FROM src
+WHERE
+   ((key = '0'
+   AND value = '8') OR (key = '1'
+   AND value = '5') OR (key = '2'
+   AND value = '6') OR (key = '3'
+   AND value = '8') OR (key = '4'
+   AND value = '1') OR (key = '5'
+   AND value = '6') OR (key = '6'
+   AND value = '1') OR (key = '7'
+   AND value = '1') OR (key = '8'
+   AND value = '1') OR (key = '9'
+   AND value = '1') OR (key = '10'
+   AND value = '3'))
+;
+
+set hive.optimize.point.lookup.extract=true;
+
+explain
+SELECT key
+FROM src
+WHERE
+   ((key = '0'
+   AND value = '8') OR (key = '1'
+   AND value = '5') OR (key = '2'
+   AND value = '6') OR (key = '3'
+   AND value = '8') OR (key = '4'
+   AND value = '1') OR (key = '5'
+   AND value = '6') OR (key = '6'
+   AND value = '1') OR (key = '7'
+   AND value = '1') OR (key = '8'
+   AND value = '1') OR (key = '9'
+   AND value = '1') OR (key = '10'
+   AND value = '3'))
+;

http://git-wip-us.apache.org/repos/asf/hive/blob/b247cac4/ql/src/test/queries/clientpositive/pointlookup2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/pointlookup2.q b/ql/src/test/queries/clientpositive/pointlookup2.q
new file mode 100644
index 0000000..31bebbb
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/pointlookup2.q
@@ -0,0 +1,51 @@
+drop table pcr_t1;
+drop table pcr_t2;
+drop table pcr_t3;
+
+create table pcr_t1 (key int, value string) partitioned by (ds string);
+insert overwrite table pcr_t1 partition (ds='2000-04-08') select * from src where key < 20 order by key;
+insert overwrite table pcr_t1 partition (ds='2000-04-09') select * from src where key < 20 order by key;
+insert overwrite table pcr_t1 partition (ds='2000-04-10') select * from src where key < 20 order by key;
+
+create table pcr_t2 (ds string, key int, value string);
+from pcr_t1
+insert overwrite table pcr_t2 select ds, key, value where ds='2000-04-08';
+from pcr_t1
+insert overwrite table pcr_t2 select ds, key, value where ds='2000-04-08' and key=2;
+
+set hive.optimize.point.lookup.min=2;
+set hive.optimize.point.lookup.extract=true;
+
+explain extended
+select key, value, ds
+from pcr_t1
+where (ds='2000-04-08' and key=1) or (ds='2000-04-09' and key=2)
+order by key, value, ds;
+
+explain extended
+select *
+from pcr_t1 t1 join pcr_t1 t2
+on t1.key=t2.key and t1.ds='2000-04-08' and t2.ds='2000-04-08'
+order by t1.key;
+
+explain extended
+select *
+from pcr_t1 t1 join pcr_t1 t2
+on t1.key=t2.key and t1.ds='2000-04-08' and t2.ds='2000-04-09'
+order by t1.key;
+
+explain extended
+select *
+from pcr_t1 t1 join pcr_t2 t2
+where (t1.ds='2000-04-08' and t2.key=1) or (t1.ds='2000-04-09' and t2.key=2)
+order by t2.key, t2.value, t1.ds;
+
+explain extended
+select *
+from pcr_t1 t1 join pcr_t2 t2
+where (t2.ds='2000-04-08' and t1.key=1) or (t2.ds='2000-04-09' and t1.key=2)
+order by t1.key, t1.value, t2.ds;
+
+drop table pcr_t1;
+drop table pcr_t2;
+drop table pcr_t3;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/b247cac4/ql/src/test/results/clientpositive/alter_partition_coltype.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/alter_partition_coltype.q.out b/ql/src/test/results/clientpositive/alter_partition_coltype.q.out
index 06515da..9fc3c8d 100644
--- a/ql/src/test/results/clientpositive/alter_partition_coltype.q.out
+++ b/ql/src/test/results/clientpositive/alter_partition_coltype.q.out
@@ -1134,15 +1134,11 @@ STAGE PLANS:
           alias: alterdynamic_part_table
           Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE Column stats: NONE
           GatherStats: false
-          Filter Operator
-            isSamplingPred: false
-            predicate: (struct(partcol1,partcol2)) IN (const struct(2,'1'), const struct(1,'__HIVE_DEFAULT_PARTITION__')) (type: boolean)
+          Select Operator
+            expressions: intcol (type: string)
+            outputColumnNames: _col0
             Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE Column stats: NONE
-            Select Operator
-              expressions: intcol (type: string)
-              outputColumnNames: _col0
-              Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE Column stats: NONE
-              ListSink
+            ListSink
 
 PREHOOK: query: select intcol from pt.alterdynamic_part_table where (partcol1='2' and partcol2='1')or (partcol1='1' and partcol2='__HIVE_DEFAULT_PARTITION__')
 PREHOOK: type: QUERY

http://git-wip-us.apache.org/repos/asf/hive/blob/b247cac4/ql/src/test/results/clientpositive/annotate_stats_filter.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/annotate_stats_filter.q.out b/ql/src/test/results/clientpositive/annotate_stats_filter.q.out
index 9e0e78a..054b573 100644
--- a/ql/src/test/results/clientpositive/annotate_stats_filter.q.out
+++ b/ql/src/test/results/clientpositive/annotate_stats_filter.q.out
@@ -678,15 +678,15 @@ STAGE PLANS:
             alias: loc_orc
             Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
-              predicate: (state) IN ('OH', 'CA') (type: boolean)
-              Statistics: Num rows: 4 Data size: 396 Basic stats: COMPLETE Column stats: COMPLETE
+              predicate: ((state = 'OH') or (state = 'CA')) (type: boolean)
+              Statistics: Num rows: 2 Data size: 204 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: state (type: string), locid (type: int), zip (type: bigint), year (type: int)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 4 Data size: 396 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 2 Data size: 204 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 4 Data size: 396 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 2 Data size: 204 Basic stats: COMPLETE Column stats: COMPLETE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/b247cac4/ql/src/test/results/clientpositive/flatten_and_or.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/flatten_and_or.q.out b/ql/src/test/results/clientpositive/flatten_and_or.q.out
index 5f25daa..9c51ff3 100644
--- a/ql/src/test/results/clientpositive/flatten_and_or.q.out
+++ b/ql/src/test/results/clientpositive/flatten_and_or.q.out
@@ -44,15 +44,15 @@ STAGE PLANS:
             alias: src
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
-              predicate: (struct(key,value)) IN (const struct('0','8'), const struct('1','5'), const struct('2','6'), const struct('3','8'), const struct('4','1'), const struct('5','6'), const struct('6','1'), const struct('7','1'), const struct('8','1'), const struct('9','1'), const struct('10','3')) (type: boolean)
-              Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+              predicate: (((key = '0') and (value = '8')) or ((key = '1') and (value = '5')) or ((key = '2') and (value = '6')) or ((key = '3') and (value = '8')) or ((key = '4') and (value = '1')) or ((key = '5') and (value = '6')) or ((key = '6') and (value = '1')) or ((key = '7') and (value = '1')) or ((key = '8') and (value = '1')) or ((key = '9') and (value = '1')) or ((key = '10') and (value = '3'))) (type: boolean)
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: key (type: string)
                 outputColumnNames: _col0
-                Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/b247cac4/ql/src/test/results/clientpositive/pcr.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/pcr.q.out b/ql/src/test/results/clientpositive/pcr.q.out
index 4c9ea77..d7c40a3 100644
--- a/ql/src/test/results/clientpositive/pcr.q.out
+++ b/ql/src/test/results/clientpositive/pcr.q.out
@@ -2475,16 +2475,16 @@ STAGE PLANS:
             GatherStats: false
             Filter Operator
               isSamplingPred: false
-              predicate: (struct(key,ds)) IN (const struct(1,'2000-04-08'), const struct(2,'2000-04-09')) (type: boolean)
-              Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
+              predicate: (((ds = '2000-04-08') and (key = 1)) or ((ds = '2000-04-09') and (key = 2))) (type: boolean)
+              Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: key (type: int), value (type: string), ds (type: string)
                 outputColumnNames: _col0, _col1, _col2
-                Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string)
                   sort order: +++
-                  Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE
                   tag: -1
                   auto parallelism: false
       Path -> Alias:
@@ -2588,13 +2588,13 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string)
           outputColumnNames: _col0, _col1, _col2
-          Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
+          Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
             NumFilesPerFileSink: 1
-            Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
+            Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE
 #### A masked pattern was here ####
             table:
                 input format: org.apache.hadoop.mapred.TextInputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/b247cac4/ql/src/test/results/clientpositive/pointlookup.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/pointlookup.q.out b/ql/src/test/results/clientpositive/pointlookup.q.out
new file mode 100644
index 0000000..7e19be4
--- /dev/null
+++ b/ql/src/test/results/clientpositive/pointlookup.q.out
@@ -0,0 +1,198 @@
+PREHOOK: query: explain
+SELECT key
+FROM src
+WHERE
+   ((key = '0'
+   AND value = '8') OR (key = '1'
+   AND value = '5') OR (key = '2'
+   AND value = '6') OR (key = '3'
+   AND value = '8') OR (key = '4'
+   AND value = '1') OR (key = '5'
+   AND value = '6') OR (key = '6'
+   AND value = '1') OR (key = '7'
+   AND value = '1') OR (key = '8'
+   AND value = '1') OR (key = '9'
+   AND value = '1') OR (key = '10'
+   AND value = '3'))
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+SELECT key
+FROM src
+WHERE
+   ((key = '0'
+   AND value = '8') OR (key = '1'
+   AND value = '5') OR (key = '2'
+   AND value = '6') OR (key = '3'
+   AND value = '8') OR (key = '4'
+   AND value = '1') OR (key = '5'
+   AND value = '6') OR (key = '6'
+   AND value = '1') OR (key = '7'
+   AND value = '1') OR (key = '8'
+   AND value = '1') OR (key = '9'
+   AND value = '1') OR (key = '10'
+   AND value = '3'))
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: src
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: (((key = '0') and (value = '8')) or ((key = '1') and (value = '5')) or ((key = '2') and (value = '6')) or ((key = '3') and (value = '8')) or ((key = '4') and (value = '1')) or ((key = '5') and (value = '6')) or ((key = '6') and (value = '1')) or ((key = '7') and (value = '1')) or ((key = '8') and (value = '1')) or ((key = '9') and (value = '1')) or ((key = '10') and (value = '3'))) (type: boolean)
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: key (type: string)
+                outputColumnNames: _col0
+                Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: explain
+SELECT key
+FROM src
+WHERE
+   ((key = '0'
+   AND value = '8') OR (key = '1'
+   AND value = '5') OR (key = '2'
+   AND value = '6') OR (key = '3'
+   AND value = '8') OR (key = '4'
+   AND value = '1') OR (key = '5'
+   AND value = '6') OR (key = '6'
+   AND value = '1') OR (key = '7'
+   AND value = '1') OR (key = '8'
+   AND value = '1') OR (key = '9'
+   AND value = '1') OR (key = '10'
+   AND value = '3'))
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+SELECT key
+FROM src
+WHERE
+   ((key = '0'
+   AND value = '8') OR (key = '1'
+   AND value = '5') OR (key = '2'
+   AND value = '6') OR (key = '3'
+   AND value = '8') OR (key = '4'
+   AND value = '1') OR (key = '5'
+   AND value = '6') OR (key = '6'
+   AND value = '1') OR (key = '7'
+   AND value = '1') OR (key = '8'
+   AND value = '1') OR (key = '9'
+   AND value = '1') OR (key = '10'
+   AND value = '3'))
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: src
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: (struct(key,value)) IN (const struct('0','8'), const struct('1','5'), const struct('2','6'), const struct('3','8'), const struct('4','1'), const struct('5','6'), const struct('6','1'), const struct('7','1'), const struct('8','1'), const struct('9','1'), const struct('10','3')) (type: boolean)
+              Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: key (type: string)
+                outputColumnNames: _col0
+                Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: explain
+SELECT key
+FROM src
+WHERE
+   ((key = '0'
+   AND value = '8') OR (key = '1'
+   AND value = '5') OR (key = '2'
+   AND value = '6') OR (key = '3'
+   AND value = '8') OR (key = '4'
+   AND value = '1') OR (key = '5'
+   AND value = '6') OR (key = '6'
+   AND value = '1') OR (key = '7'
+   AND value = '1') OR (key = '8'
+   AND value = '1') OR (key = '9'
+   AND value = '1') OR (key = '10'
+   AND value = '3'))
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+SELECT key
+FROM src
+WHERE
+   ((key = '0'
+   AND value = '8') OR (key = '1'
+   AND value = '5') OR (key = '2'
+   AND value = '6') OR (key = '3'
+   AND value = '8') OR (key = '4'
+   AND value = '1') OR (key = '5'
+   AND value = '6') OR (key = '6'
+   AND value = '1') OR (key = '7'
+   AND value = '1') OR (key = '8'
+   AND value = '1') OR (key = '9'
+   AND value = '1') OR (key = '10'
+   AND value = '3'))
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: src
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: ((value) IN ('1', '3', '5', '6', '8') and (struct(key,value)) IN (const struct('0','8'), const struct('1','5'), const struct('2','6'), const struct('3','8'), const struct('4','1'), const struct('5','6'), const struct('6','1'), const struct('7','1'), const struct('8','1'), const struct('9','1'), const struct('10','3'))) (type: boolean)
+              Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: key (type: string)
+                outputColumnNames: _col0
+                Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+