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/07/31 02:43:01 UTC

[01/43] hive git commit: HIVE-11141 : Improve RuleRegExp when the Expression node stack gets huge (Hari Subramaniyan, reviewed by Laljo John Pullokkaran, Jesus Camacho Rodriguez)

Repository: hive
Updated Branches:
  refs/heads/spark 537114b96 -> 714b3db65


HIVE-11141 : Improve RuleRegExp when the Expression node stack gets huge (Hari Subramaniyan, reviewed by Laljo John Pullokkaran, 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/0ad4f717
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/0ad4f717
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/0ad4f717

Branch: refs/heads/spark
Commit: 0ad4f717a7c06bd7bbd90d4b3e861ba1e25d14b7
Parents: 941610f
Author: Hari Subramaniyan <ha...@apache.org>
Authored: Mon Jul 20 17:17:03 2015 -0700
Committer: Hari Subramaniyan <ha...@apache.org>
Committed: Mon Jul 20 17:17:03 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hive/ql/lib/RuleRegExp.java   | 191 ++++++++++++++++++-
 .../hadoop/hive/ql/lib/TestRuleRegExp.java      | 118 ++++++++++++
 2 files changed, 300 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/0ad4f717/ql/src/java/org/apache/hadoop/hive/ql/lib/RuleRegExp.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lib/RuleRegExp.java b/ql/src/java/org/apache/hadoop/hive/ql/lib/RuleRegExp.java
index ddc96c2..c88ed68 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lib/RuleRegExp.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lib/RuleRegExp.java
@@ -18,6 +18,9 @@
 
 package org.apache.hadoop.hive.ql.lib;
 
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
 import java.util.Stack;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
@@ -31,7 +34,54 @@ import org.apache.hadoop.hive.ql.parse.SemanticException;
 public class RuleRegExp implements Rule {
 
   private final String ruleName;
-  private final Pattern pattern;
+  private final Pattern patternWithWildCardChar;
+  private final String patternWithoutWildCardChar;
+  private String[] patternORWildChar;
+  private static final Set<Character> wildCards = new HashSet<Character>(Arrays.asList(
+    '[', '^', '$', '*', ']', '+', '|', '(', '\\', '.', '?', ')', '&'));
+
+  /**
+   * The function iterates through the list of wild card characters and sees if
+   * this regular expression contains a wild card character.
+   *
+   * @param pattern
+   *          pattern expressed as a regular Expression
+   */
+  private static boolean patternHasWildCardChar(String pattern) {
+    if (pattern == null) {
+      return false;
+    }
+    for (char pc : pattern.toCharArray()) {
+      if (wildCards.contains(pc)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * The function iterates through the list of wild card characters and sees if
+   * this regular expression contains  only the given char as wild card character.
+   *
+   * @param pattern
+   *          pattern expressed as a regular Expression
+   * @param wcc
+   *          wild card character
+   */
+  private static boolean patternHasOnlyWildCardChar(String pattern, char wcc) {
+    if (pattern == null) {
+      return false;
+    }
+    boolean ret = true;
+    boolean hasWildCard = false;
+    for (char pc : pattern.toCharArray()) {
+      if (wildCards.contains(pc)) {
+        hasWildCard = true;
+        ret = ret && (pc == wcc);
+      }
+    }
+    return ret && hasWildCard;
+  }
 
   /**
    * The rule specified by the regular expression. Note that, the regular
@@ -46,33 +96,156 @@ public class RuleRegExp implements Rule {
    **/
   public RuleRegExp(String ruleName, String regExp) {
     this.ruleName = ruleName;
-    pattern = Pattern.compile(regExp);
+
+    if (patternHasWildCardChar(regExp)) {
+      if (patternHasOnlyWildCardChar(regExp, '|')) {
+          this.patternWithWildCardChar = null;
+          this.patternWithoutWildCardChar = null;
+          this.patternORWildChar = regExp.split("\\|");
+      } else {
+        this.patternWithWildCardChar = Pattern.compile(regExp);
+        this.patternWithoutWildCardChar = null;
+        this.patternORWildChar = null;
+      }
+    } else {
+      this.patternWithWildCardChar = null;
+      this.patternWithoutWildCardChar = regExp;
+      this.patternORWildChar = null;
+    }
   }
 
   /**
-   * This function returns the cost of the rule for the specified stack. Lower
-   * the cost, the better the rule is matched
-   * 
+   * This function returns the cost of the rule for the specified stack when the pattern
+   * matched for has no wildcard character in it. The function expects patternWithoutWildCardChar
+   * to be not null.
    * @param stack
    *          Node stack encountered so far
    * @return cost of the function
    * @throws SemanticException
    */
-  @Override
-  public int cost(Stack<Node> stack) throws SemanticException {
+  private int costPatternWithoutWildCardChar(Stack<Node> stack) throws SemanticException {
     int numElems = (stack != null ? stack.size() : 0);
+    String name = new String("");
+    int patLen = patternWithoutWildCardChar.length();
+
+    for (int pos = numElems - 1; pos >= 0; pos--) {
+        name = stack.get(pos).getName() + "%" + name;
+      if (name.length() >= patLen) {
+        if (patternWithoutWildCardChar.equals(name)) {
+          return patLen;
+        } else {
+          return -1;
+        }
+      }
+    }
+    return -1;
+  }
+
+  /**
+   * This function returns the cost of the rule for the specified stack when the pattern
+   * matched for has only OR wildcard character in it. The function expects patternORWildChar
+   * to be not null.
+   * @param stack
+   *          Node stack encountered so far
+   * @return cost of the function
+   * @throws SemanticException
+   */
+  private int costPatternWithORWildCardChar(Stack<Node> stack) throws SemanticException {
+    int numElems = (stack != null ? stack.size() : 0);
+    for (String pattern : patternORWildChar) {
+      String name = new String("");
+      int patLen = pattern.length();
+
+      for (int pos = numElems - 1; pos >= 0; pos--) {
+        name = stack.get(pos).getName() + "%" + name;
+        if (name.length() >= patLen) {
+          if (pattern.equals(name)) {
+            return patLen;
+          } else {
+            break;
+          }
+        }
+      }
+    }
+    return -1;
+  }
+
+  /**
+   * This function returns the cost of the rule for the specified stack when the pattern
+   * matched for has wildcard character in it. The function expects patternWithWildCardChar
+   * to be not null.
+   *
+   * @param stack
+   *          Node stack encountered so far
+   * @return cost of the function
+   * @throws SemanticException
+   */
+  private int costPatternWithWildCardChar(Stack<Node> stack) throws SemanticException {
+	int numElems = (stack != null ? stack.size() : 0);
     String name = "";
+    Matcher m = patternWithWildCardChar.matcher("");
     for (int pos = numElems - 1; pos >= 0; pos--) {
       name = stack.get(pos).getName() + "%" + name;
-      Matcher m = pattern.matcher(name);
+      m.reset(name);
       if (m.matches()) {
-        return m.group().length();
+        return name.length();
       }
     }
     return -1;
   }
 
   /**
+   * Returns true if the rule pattern is valid and has wild character in it.
+   */
+  boolean rulePatternIsValidWithWildCardChar() {
+    return patternWithoutWildCardChar == null && patternWithWildCardChar != null && this.patternORWildChar == null;
+  }
+
+  /**
+   * Returns true if the rule pattern is valid and has wild character in it.
+   */
+  boolean rulePatternIsValidWithoutWildCardChar() {
+    return patternWithWildCardChar == null && patternWithoutWildCardChar != null && this.patternORWildChar == null;
+  }
+
+  /**
+   * Returns true if the rule pattern is valid and has wild character in it.
+   */
+  boolean rulePatternIsValidWithORWildCardChar() {
+    return patternWithoutWildCardChar == null && patternWithWildCardChar == null && this.patternORWildChar != null;
+  }
+
+  /**
+   * This function returns the cost of the rule for the specified stack. Lower
+   * the cost, the better the rule is matched
+   *
+   * @param stack
+   *          Node stack encountered so far
+   * @return cost of the function
+   * @throws SemanticException
+   */
+  @Override
+  public int cost(Stack<Node> stack) throws SemanticException {
+    if (rulePatternIsValidWithoutWildCardChar()) {
+      return costPatternWithoutWildCardChar(stack);
+    }
+    if (rulePatternIsValidWithWildCardChar()) {
+      return costPatternWithWildCardChar(stack);
+    }
+    if (rulePatternIsValidWithORWildCardChar()) {
+      return costPatternWithORWildCardChar(stack);
+    }
+    // If we reached here, either :
+    // 1. patternWithWildCardChar and patternWithoutWildCardChar are both nulls.
+    // 2. patternWithWildCardChar and patternWithoutWildCardChar are both not nulls.
+    // This is an internal error and we should not let this happen, so throw an exception.
+    throw new SemanticException (
+      "Rule pattern is invalid for " + getName() + " : patternWithWildCardChar = " +
+      patternWithWildCardChar + " patternWithoutWildCardChar = " +
+      patternWithoutWildCardChar);
+  }
+
+  /**
    * @return the name of the Node
    **/
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/0ad4f717/ql/src/test/org/apache/hadoop/hive/ql/lib/TestRuleRegExp.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/lib/TestRuleRegExp.java b/ql/src/test/org/apache/hadoop/hive/ql/lib/TestRuleRegExp.java
new file mode 100644
index 0000000..f06d0df
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/lib/TestRuleRegExp.java
@@ -0,0 +1,118 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.lib;
+
+import static org.junit.Assert.*;
+
+import java.util.List;
+import java.util.Stack;
+
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.hive.ql.exec.FilterOperator;
+import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
+import org.apache.hadoop.hive.ql.exec.SelectOperator;
+import org.apache.hadoop.hive.ql.exec.TableScanOperator;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.junit.Test;
+
+public class TestRuleRegExp {
+
+  public class TestNode implements Node {
+    private String name;
+
+    TestNode (String name) {
+      this.name = name;
+    }
+
+    @Override
+    public List<? extends Node> getChildren() {
+      return null;
+    }
+
+    @Override
+    public String getName() {
+      return name;
+    }
+  }
+
+  @Test
+  public void testPatternWithoutWildCardChar() {
+    String patternStr =
+      ReduceSinkOperator.getOperatorName() + "%" +
+      SelectOperator.getOperatorName() + "%" +
+      FileSinkOperator.getOperatorName() + "%";
+    RuleRegExp rule1 = new RuleRegExp("R1", patternStr);
+    assertEquals(rule1.rulePatternIsValidWithoutWildCardChar(), true);
+    assertEquals(rule1.rulePatternIsValidWithWildCardChar(), false);
+    // positive test
+    Stack<Node> ns1 = new Stack<Node>();
+    ns1.push(new TestNode(ReduceSinkOperator.getOperatorName()));
+    ns1.push(new TestNode(SelectOperator.getOperatorName()));
+    ns1.push(new TestNode(FileSinkOperator.getOperatorName()));
+    try {
+      assertEquals(rule1.cost(ns1), patternStr.length());
+    } catch (SemanticException e) {
+      fail(e.getMessage());
+	}
+    // negative test
+    Stack<Node> ns2 = new Stack<Node>();
+    ns2.push(new TestNode(ReduceSinkOperator.getOperatorName()));
+    ns1.push(new TestNode(TableScanOperator.getOperatorName()));
+    ns1.push(new TestNode(FileSinkOperator.getOperatorName()));
+    try {
+      assertEquals(rule1.cost(ns2), -1);
+    } catch (SemanticException e) {
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void testPatternWithWildCardChar() {
+    RuleRegExp rule1 =  new RuleRegExp("R1",
+      "(" + TableScanOperator.getOperatorName() + "%"
+      + FilterOperator.getOperatorName() + "%)|("
+      + TableScanOperator.getOperatorName() + "%"
+      + FileSinkOperator.getOperatorName() + "%)");
+    assertEquals(rule1.rulePatternIsValidWithoutWildCardChar(), false);
+    assertEquals(rule1.rulePatternIsValidWithWildCardChar(), true);
+    // positive test
+    Stack<Node> ns1 = new Stack<Node>();
+    ns1.push(new TestNode(TableScanOperator.getOperatorName()));
+    ns1.push(new TestNode(FilterOperator.getOperatorName()));
+    Stack<Node> ns2 = new Stack<Node>();
+    ns2.push(new TestNode(TableScanOperator.getOperatorName()));
+    ns2.push(new TestNode(FileSinkOperator.getOperatorName()));
+    try {
+      assertNotEquals(rule1.cost(ns1), -1);
+      assertNotEquals(rule1.cost(ns2), -1);
+    } catch (SemanticException e) {
+      fail(e.getMessage());
+	}
+    // negative test
+    Stack<Node> ns3 = new Stack<Node>();
+    ns3.push(new TestNode(ReduceSinkOperator.getOperatorName()));
+    ns3.push(new TestNode(ReduceSinkOperator.getOperatorName()));
+    ns3.push(new TestNode(FileSinkOperator.getOperatorName()));
+    try {
+      assertEquals(rule1.cost(ns3), -1);
+    } catch (SemanticException e) {
+      fail(e.getMessage());
+    }
+  }
+
+}


[10/43] hive git commit: HIVE-11210. Remove dependency on HiveConf from Orc reader and writer.

Posted by xu...@apache.org.
HIVE-11210. Remove dependency on HiveConf from Orc reader and writer.


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

Branch: refs/heads/spark
Commit: bab3ee31eec98ff28366949461e004d1386100f6
Parents: 04d54f6
Author: Owen O'Malley <om...@apache.org>
Authored: Wed Jul 8 15:02:49 2015 -0700
Committer: Owen O'Malley <om...@apache.org>
Committed: Tue Jul 21 14:07:39 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hive/ql/io/orc/MemoryManager.java    |   4 +-
 .../apache/hadoop/hive/ql/io/orc/OrcConf.java   | 134 +++++++++++++++++++
 .../apache/hadoop/hive/ql/io/orc/OrcFile.java   |  40 +++---
 .../apache/hadoop/hive/ql/io/orc/Reader.java    |  34 ++++-
 .../hadoop/hive/ql/io/orc/RecordReaderImpl.java |  33 ++---
 .../hadoop/hive/ql/io/orc/WriterImpl.java       |  27 ++--
 6 files changed, 212 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/bab3ee31/ql/src/java/org/apache/hadoop/hive/ql/io/orc/MemoryManager.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/MemoryManager.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/MemoryManager.java
index 6432d6e..0347a1c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/MemoryManager.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/MemoryManager.java
@@ -22,7 +22,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.conf.HiveConf;
 
 import com.google.common.base.Preconditions;
 
@@ -91,8 +90,7 @@ class MemoryManager {
    *             pool.
    */
   MemoryManager(Configuration conf) {
-    HiveConf.ConfVars poolVar = HiveConf.ConfVars.HIVE_ORC_FILE_MEMORY_POOL;
-    double maxLoad = conf.getFloat(poolVar.varname, poolVar.defaultFloatVal);
+    double maxLoad = OrcConf.MEMORY_POOL.getDouble(conf);
     totalMemoryPool = Math.round(ManagementFactory.getMemoryMXBean().
         getHeapMemoryUsage().getMax() * maxLoad);
     ownerLock.lock();

http://git-wip-us.apache.org/repos/asf/hive/blob/bab3ee31/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcConf.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcConf.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcConf.java
new file mode 100644
index 0000000..aeb0ec1
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcConf.java
@@ -0,0 +1,134 @@
+/**
+ * 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.io.orc;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Define the configuration properties that Orc understands.
+ */
+public enum OrcConf {
+  STRIPE_SIZE("hive.exec.orc.default.stripe.size",
+      64L * 1024 * 1024,
+      "Define the default ORC stripe size, in bytes."),
+  BLOCK_SIZE("hive.exec.orc.default.block.size", 256L * 1024 * 1024,
+      "Define the default file system block size for ORC files."),
+  ROW_INDEX_STRIDE("hive.exec.orc.default.row.index.stride", 10000,
+      "Define the default ORC index stride in number of rows. (Stride is the\n"+
+          " number of rows n index entry represents.)"),
+  BUFFER_SIZE("hive.exec.orc.default.buffer.size", 256 * 1024,
+      "Define the default ORC buffer size, in bytes."),
+  BLOCK_PADDING("hive.exec.orc.default.block.padding", true,
+      "Define the default block padding, which pads stripes to the HDFS\n" +
+          " block boundaries."),
+  COMPRESS("hive.exec.orc.default.compress", "ZLIB",
+      "Define the default compression codec for ORC file"),
+  WRITE_FORMAT("hive.exec.orc.write.format", null,
+      "Define the version of the file to write. Possible values are 0.11 and\n"+
+          " 0.12. If this parameter is not defined, ORC will use the run\n" +
+          " length encoding (RLE) introduced in Hive 0.12. Any value other\n" +
+          " than 0.11 results in the 0.12 encoding."),
+  ENCODING_STRATEGY("hive.exec.orc.encoding.strategy", "SPEED",
+      "Define the encoding strategy to use while writing data. Changing this\n"+
+          "will only affect the light weight encoding for integers. This\n" +
+          "flag will not change the compression level of higher level\n" +
+          "compression codec (like ZLIB)."),
+  COMPRESSION_STRATEGY("hive.exec.orc.compression.strategy", "SPEED",
+      "Define the compression strategy to use while writing data.\n" +
+          "This changes the compression level of higher level compression\n" +
+          "codec (like ZLIB)."),
+  BLOCK_PADDING_TOLERANCE("hive.exec.orc.block.padding.tolerance",
+      0.05,
+      "Define the tolerance for block padding as a decimal fraction of\n" +
+          "stripe size (for example, the default value 0.05 is 5% of the\n" +
+          "stripe size). For the defaults of 64Mb ORC stripe and 256Mb HDFS\n" +
+          "blocks, the default block padding tolerance of 5% will\n" +
+          "reserve a maximum of 3.2Mb for padding within the 256Mb block.\n" +
+          "In that case, if the available size within the block is more than\n"+
+          "3.2Mb, a new smaller stripe will be inserted to fit within that\n" +
+          "space. This will make sure that no stripe written will block\n" +
+          " boundaries and cause remote reads within a node local task."),
+  BLOOM_FILTER_FPP("orc.default.bloom.fpp", 0.05,
+      "Define the default false positive probability for bloom filters."),
+  USE_ZEROCOPY("hive.exec.orc.zerocopy", false,
+      "Use zerocopy reads with ORC. (This requires Hadoop 2.3 or later.)"),
+  SKIP_CORRUPT_DATA("hive.exec.orc.skip.corrupt.data", false,
+      "If ORC reader encounters corrupt data, this value will be used to\n" +
+          "determine whether to skip the corrupt data or throw exception.\n" +
+          "The default behavior is to throw exception."),
+  MEMORY_POOL("hive.exec.orc.memory.pool", 0.5,
+      "Maximum fraction of heap that can be used by ORC file writers"),
+  DICTIONARY_KEY_SIZE_THRESHOLD("hive.exec.orc.dictionary.key.size.threshold",
+      0.8,
+      "If the number of keys in a dictionary is greater than this fraction\n" +
+          "of the total number of non-null rows, turn off dictionary\n" +
+          "encoding.  Use 1 to always use dictionary encoding."),
+  ROW_INDEX_STRIDE_DICTIONARY_CHECK("hive.orc.row.index.stride.dictionary.check",
+      true,
+      "If enabled dictionary check will happen after first row index stride\n" +
+          "(default 10000 rows) else dictionary check will happen before\n" +
+          "writing first stripe. In both cases, the decision to use\n" +
+          "dictionary or not will be retained thereafter."),
+  ;
+
+  private final String attribute;
+  private final Object defaultValue;
+  private final String description;
+
+  OrcConf(String attribute, Object defaultValue, String description) {
+    this.attribute = attribute;
+    this.defaultValue = defaultValue;
+    this.description = description;
+  }
+
+  public String getAttribute() {
+    return attribute;
+  }
+
+  public Object getDefaultValue() {
+    return defaultValue;
+  }
+
+  public String getDescription() {
+    return description;
+  }
+
+  public long getLong(Configuration conf) {
+    return conf.getLong(attribute, ((Number) defaultValue).longValue());
+  }
+
+  public String getString(Configuration conf) {
+    return conf.get(attribute, (String) defaultValue);
+  }
+
+  public boolean getBoolean(Configuration conf) {
+    if (conf == null) {
+      return (Boolean) defaultValue;
+    }
+    return conf.getBoolean(attribute, (Boolean) defaultValue);
+  }
+
+  public double getDouble(Configuration conf) {
+    String str = conf.get(attribute);
+    if (str == null) {
+      return ((Number) defaultValue).doubleValue();
+    }
+    return Double.parseDouble(str);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/bab3ee31/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFile.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFile.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFile.java
index 4e2bd6a..976a84b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFile.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFile.java
@@ -18,21 +18,11 @@
 
 package org.apache.hadoop.hive.ql.io.orc;
 
-import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_ORC_DEFAULT_BLOCK_PADDING;
-import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_ORC_DEFAULT_BLOCK_SIZE;
-import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_ORC_DEFAULT_BUFFER_SIZE;
-import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_ORC_DEFAULT_COMPRESS;
-import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_ORC_DEFAULT_ROW_INDEX_STRIDE;
-import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_ORC_DEFAULT_STRIPE_SIZE;
-import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_ORC_WRITE_FORMAT;
-
 import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.ql.io.filters.BloomFilterIO;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 
 /**
@@ -264,44 +254,44 @@ public final class OrcFile {
     private WriterCallback callback;
     private EncodingStrategy encodingStrategy;
     private CompressionStrategy compressionStrategy;
-    private float paddingTolerance;
+    private double paddingTolerance;
     private String bloomFilterColumns;
     private double bloomFilterFpp;
 
     WriterOptions(Configuration conf) {
       configuration = conf;
       memoryManagerValue = getMemoryManager(conf);
-      stripeSizeValue = HiveConf.getLongVar(conf, HIVE_ORC_DEFAULT_STRIPE_SIZE);
-      blockSizeValue = HiveConf.getLongVar(conf, HIVE_ORC_DEFAULT_BLOCK_SIZE);
-      rowIndexStrideValue = HiveConf.getIntVar(conf, HIVE_ORC_DEFAULT_ROW_INDEX_STRIDE);
-      bufferSizeValue = HiveConf.getIntVar(conf, HIVE_ORC_DEFAULT_BUFFER_SIZE);
-      blockPaddingValue = HiveConf.getBoolVar(conf, HIVE_ORC_DEFAULT_BLOCK_PADDING);
-      compressValue = CompressionKind.valueOf(HiveConf.getVar(conf, HIVE_ORC_DEFAULT_COMPRESS));
-      String versionName = HiveConf.getVar(conf, HIVE_ORC_WRITE_FORMAT);
+      stripeSizeValue = OrcConf.STRIPE_SIZE.getLong(conf);
+      blockSizeValue = OrcConf.BLOCK_SIZE.getLong(conf);
+      rowIndexStrideValue =
+          (int) OrcConf.ROW_INDEX_STRIDE.getLong(conf);
+      bufferSizeValue = (int) OrcConf.BUFFER_SIZE.getLong(conf);
+      blockPaddingValue = OrcConf.BLOCK_PADDING.getBoolean(conf);
+      compressValue =
+          CompressionKind.valueOf(OrcConf.COMPRESS.getString(conf));
+      String versionName = OrcConf.WRITE_FORMAT.getString(conf);
       if (versionName == null) {
         versionValue = Version.CURRENT;
       } else {
         versionValue = Version.byName(versionName);
       }
-      String enString =
-          conf.get(HiveConf.ConfVars.HIVE_ORC_ENCODING_STRATEGY.varname);
+      String enString = OrcConf.ENCODING_STRATEGY.getString(conf);
       if (enString == null) {
         encodingStrategy = EncodingStrategy.SPEED;
       } else {
         encodingStrategy = EncodingStrategy.valueOf(enString);
       }
 
-      String compString = conf
-          .get(HiveConf.ConfVars.HIVE_ORC_COMPRESSION_STRATEGY.varname);
+      String compString = OrcConf.COMPRESSION_STRATEGY.getString(conf);
       if (compString == null) {
         compressionStrategy = CompressionStrategy.SPEED;
       } else {
         compressionStrategy = CompressionStrategy.valueOf(compString);
       }
 
-      paddingTolerance = conf.getFloat(HiveConf.ConfVars.HIVE_ORC_BLOCK_PADDING_TOLERANCE.varname,
-          HiveConf.ConfVars.HIVE_ORC_BLOCK_PADDING_TOLERANCE.defaultFloatVal);
-      bloomFilterFpp = BloomFilterIO.DEFAULT_FPP;
+      paddingTolerance =
+          OrcConf.BLOCK_PADDING_TOLERANCE.getDouble(conf);
+      bloomFilterFpp = OrcConf.BLOOM_FILTER_FPP.getDouble(conf);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/hive/blob/bab3ee31/ql/src/java/org/apache/hadoop/hive/ql/io/orc/Reader.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/Reader.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/Reader.java
index 1f29085..6f4f013 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/Reader.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/Reader.java
@@ -147,6 +147,8 @@ public interface Reader {
     private long length = Long.MAX_VALUE;
     private SearchArgument sarg = null;
     private String[] columnNames = null;
+    private Boolean useZeroCopy = null;
+    private Boolean skipCorruptRecords = null;
 
     /**
      * Set the list of columns to read.
@@ -174,7 +176,7 @@ public interface Reader {
      * Set search argument for predicate push down.
      * @param sarg the search argument
      * @param columnNames the column names for
-     * @return
+     * @return this
      */
     public Options searchArgument(SearchArgument sarg, String[] columnNames) {
       this.sarg = sarg;
@@ -182,6 +184,26 @@ public interface Reader {
       return this;
     }
 
+    /**
+     * Set whether to use zero copy from HDFS.
+     * @param value the new zero copy flag
+     * @return this
+     */
+    public Options useZeroCopy(boolean value) {
+      this.useZeroCopy = value;
+      return this;
+    }
+
+    /**
+     * Set whether to skip corrupt records.
+     * @param value the new skip corrupt records flag
+     * @return this
+     */
+    public Options skipCorruptRecords(boolean value) {
+      this.skipCorruptRecords = value;
+      return this;
+    }
+
     public boolean[] getInclude() {
       return include;
     }
@@ -210,6 +232,14 @@ public interface Reader {
       return result;
     }
 
+    public Boolean getUseZeroCopy() {
+      return useZeroCopy;
+    }
+
+    public Boolean getSkipCorruptRecords() {
+      return skipCorruptRecords;
+    }
+
     public Options clone() {
       Options result = new Options();
       result.include = include;
@@ -217,6 +247,8 @@ public interface Reader {
       result.length = length;
       result.sarg = sarg;
       result.columnNames = columnNames;
+      result.useZeroCopy = useZeroCopy;
+      result.skipCorruptRecords = skipCorruptRecords;
       return result;
     }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/bab3ee31/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
index beaf231..4f79e37 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.hive.ql.io.orc;
 
-import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_ORC_ZEROCOPY;
-
 import java.io.IOException;
 import java.math.BigDecimal;
 import java.nio.ByteBuffer;
@@ -41,7 +39,6 @@ import org.apache.hadoop.hive.common.DiskRange;
 import org.apache.hadoop.hive.common.DiskRangeList;
 import org.apache.hadoop.hive.common.DiskRangeList.DiskRangeListCreateHelper;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
-import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.io.filters.BloomFilterIO;
@@ -153,15 +150,15 @@ class RecordReaderImpl implements RecordReader {
   }
 
   protected RecordReaderImpl(List<StripeInformation> stripes,
-                   FileSystem fileSystem,
-                   Path path,
-                   Reader.Options options,
-                   List<OrcProto.Type> types,
-                   CompressionCodec codec,
-                   int bufferSize,
-                   long strideRate,
-                   Configuration conf
-                   ) throws IOException {
+                             FileSystem fileSystem,
+                             Path path,
+                             Reader.Options options,
+                             List<OrcProto.Type> types,
+                             CompressionCodec codec,
+                             int bufferSize,
+                             long strideRate,
+                             Configuration conf
+                             ) throws IOException {
     this.path = path;
     this.file = fileSystem.open(path);
     this.codec = codec;
@@ -192,13 +189,19 @@ class RecordReaderImpl implements RecordReader {
       }
     }
 
-    final boolean zeroCopy = (conf != null)
-        && (HiveConf.getBoolVar(conf, HIVE_ORC_ZEROCOPY));
+    Boolean zeroCopy = options.getUseZeroCopy();
+    if (zeroCopy == null) {
+      zeroCopy = OrcConf.USE_ZEROCOPY.getBoolean(conf);
+    }
     zcr = zeroCopy ? RecordReaderUtils.createZeroCopyShim(file, codec, pool) : null;
 
     firstRow = skippedRows;
     totalRowCount = rows;
-    boolean skipCorrupt = HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_ORC_SKIP_CORRUPT_DATA);
+    Boolean skipCorrupt = options.getSkipCorruptRecords();
+    if (skipCorrupt == null) {
+      skipCorrupt = OrcConf.SKIP_CORRUPT_DATA.getBoolean(conf);
+    }
+
     reader = RecordReaderFactory.createTreeReader(0, conf, types, included, skipCorrupt);
     indexes = new OrcProto.RowIndex[types.size()];
     bloomFilterIndices = new OrcProto.BloomFilterIndex[types.size()];

http://git-wip-us.apache.org/repos/asf/hive/blob/bab3ee31/ql/src/java/org/apache/hadoop/hive/ql/io/orc/WriterImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/WriterImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/WriterImpl.java
index ee6110e..7aa8d65 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/WriterImpl.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/WriterImpl.java
@@ -40,7 +40,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.JavaUtils;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
-import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.io.IOConstants;
 import org.apache.hadoop.hive.ql.io.filters.BloomFilterIO;
 import org.apache.hadoop.hive.ql.io.orc.CompressionCodec.Modifier;
@@ -127,7 +126,7 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
   private final boolean addBlockPadding;
   private final int bufferSize;
   private final long blockSize;
-  private final float paddingTolerance;
+  private final double paddingTolerance;
   // the streams that make up the current stripe
   private final Map<StreamName, BufferedStream> streams =
     new TreeMap<StreamName, BufferedStream>();
@@ -176,7 +175,7 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
       OrcFile.WriterCallback callback,
       EncodingStrategy encodingStrategy,
       CompressionStrategy compressionStrategy,
-      float paddingTolerance,
+      double paddingTolerance,
       long blockSizeValue,
       String bloomFilterColumnNames,
       double bloomFilterFpp) throws IOException {
@@ -315,8 +314,7 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
   // the assumption is only one ORC writer open at a time, which holds true for
   // most of the cases. HIVE-6455 forces single writer case.
   private long getMemoryAvailableForORC() {
-    HiveConf.ConfVars poolVar = HiveConf.ConfVars.HIVE_ORC_FILE_MEMORY_POOL;
-    double maxLoad = conf.getFloat(poolVar.varname, poolVar.defaultFloatVal);
+    double maxLoad = OrcConf.MEMORY_POOL.getDouble(conf);
     long totalMemoryPool = Math.round(ManagementFactory.getMemoryMXBean().
         getHeapMemoryUsage().getMax() * maxLoad);
     return totalMemoryPool;
@@ -1178,7 +1176,7 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
     private final List<Long> rowIndexValueCount = new ArrayList<Long>();
     // If the number of keys in a dictionary is greater than this fraction of
     //the total number of non-null rows, turn off dictionary encoding
-    private final float dictionaryKeySizeThreshold;
+    private final double dictionaryKeySizeThreshold;
     private boolean useDictionaryEncoding = true;
     private boolean isDirectV2 = true;
     private boolean doneDictionaryCheck;
@@ -1202,14 +1200,11 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
       directStreamOutput = writer.createStream(id, OrcProto.Stream.Kind.DATA);
       directLengthOutput = createIntegerWriter(writer.createStream(id,
           OrcProto.Stream.Kind.LENGTH), false, isDirectV2, writer);
-      dictionaryKeySizeThreshold = writer.getConfiguration().getFloat(
-          HiveConf.ConfVars.HIVE_ORC_DICTIONARY_KEY_SIZE_THRESHOLD.varname,
-          HiveConf.ConfVars.HIVE_ORC_DICTIONARY_KEY_SIZE_THRESHOLD.
-              defaultFloatVal);
-      strideDictionaryCheck = writer.getConfiguration().getBoolean(
-          HiveConf.ConfVars.HIVE_ORC_ROW_INDEX_STRIDE_DICTIONARY_CHECK.varname,
-          HiveConf.ConfVars.HIVE_ORC_ROW_INDEX_STRIDE_DICTIONARY_CHECK.
-            defaultBoolVal);
+      Configuration conf = writer.getConfiguration();
+      dictionaryKeySizeThreshold =
+          OrcConf.DICTIONARY_KEY_SIZE_THRESHOLD.getDouble(conf);
+      strideDictionaryCheck =
+          OrcConf.ROW_INDEX_STRIDE_DICTIONARY_CHECK.getBoolean(conf);
       doneDictionaryCheck = false;
     }
 
@@ -2189,8 +2184,8 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
         // and user specified padding tolerance. Since stripe size can overflow
         // the default stripe size we should apply this correction to avoid
         // writing portion of last stripe to next hdfs block.
-        float correction = overflow > 0 ? (float) overflow
-            / (float) adjustedStripeSize : 0.0f;
+        double correction = overflow > 0 ? (double) overflow
+            / (double) adjustedStripeSize : 0.0;
 
         // correction should not be greater than user specified padding
         // tolerance


[43/43] hive git commit: HIVE-10863: Merge master to Spark branch 7/29/2015 [Spark Branch] (reviewed by Chao)

Posted by xu...@apache.org.
HIVE-10863: Merge master to Spark branch 7/29/2015 [Spark Branch] (reviewed by Chao)


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

Branch: refs/heads/spark
Commit: 714b3db65d41dd96db59ca1b9a6d1b6a4613072e
Parents: 537114b 7df9d7a
Author: xzhang <xz...@xzdt>
Authored: Thu Jul 30 17:41:17 2015 -0700
Committer: xzhang <xz...@xzdt>
Committed: Thu Jul 30 17:41:17 2015 -0700

----------------------------------------------------------------------
 .../java/org/apache/hive/beeline/BeeLine.java   |    5 +-
 .../org/apache/hive/beeline/BeeLineOpts.java    |    9 +
 .../apache/hive/beeline/DatabaseConnection.java |    9 +
 beeline/src/main/resources/BeeLine.properties   |    2 +
 .../hadoop/hive/cli/TestOptionsProcessor.java   |    1 -
 common/pom.xml                                  |    5 +
 .../hadoop/hive/common/ValidReadTxnList.java    |    2 +-
 .../hadoop/hive/common/type/HiveDecimal.java    |  306 --
 .../org/apache/hadoop/hive/conf/HiveConf.java   |    3 +
 .../hive/hcatalog/mapreduce/PartInfo.java       |   32 +-
 .../hive/hcatalog/mapreduce/SpecialCases.java   |    8 +-
 .../mapreduce/TestHCatMultiOutputFormat.java    |    6 +-
 .../hive/hcatalog/streaming/mutate/package.html |   31 +-
 .../mutate/worker/CreatePartitionHelper.java    |   83 -
 .../mutate/worker/MetaStorePartitionHelper.java |  102 +
 .../mutate/worker/MutatorCoordinator.java       |   21 +-
 .../worker/MutatorCoordinatorBuilder.java       |   41 +-
 .../mutate/worker/PartitionHelper.java          |   17 +
 .../mutate/worker/WarehousePartitionHelper.java |   69 +
 .../worker/TestMetaStorePartitionHelper.java    |  112 +
 .../mutate/worker/TestMutatorCoordinator.java   |   40 +-
 .../worker/TestWarehousePartitionHelper.java    |   57 +
 .../antlr4/org/apache/hive/hplsql/Hplsql.g4     |   78 +-
 .../main/java/org/apache/hive/hplsql/Conn.java  |   16 +-
 .../main/java/org/apache/hive/hplsql/Exec.java  |  118 +-
 .../java/org/apache/hive/hplsql/Expression.java |   28 +
 .../main/java/org/apache/hive/hplsql/Query.java |   71 +
 .../main/java/org/apache/hive/hplsql/Stmt.java  |  129 +-
 .../main/java/org/apache/hive/hplsql/Utils.java |    7 +
 .../main/java/org/apache/hive/hplsql/Var.java   |   27 +-
 .../apache/hive/hplsql/functions/Function.java  |   12 +-
 hplsql/src/main/resources/hplsql-site.xml       |   95 +
 .../org/apache/hive/hplsql/TestHplsqlLocal.java |   26 +-
 .../db/create_procedure_return_cursor.sql       |   53 +
 .../db/create_procedure_return_cursor2.sql      |   59 +
 .../src/test/queries/db/cursor_attributes.sql   |   60 +
 hplsql/src/test/queries/db/select_into.sql      |   17 +
 .../src/test/queries/db/set_current_schema.sql  |    6 +
 hplsql/src/test/queries/db/sys_refcursor.sql    |   65 +
 hplsql/src/test/queries/db/use.sql              |    2 +
 hplsql/src/test/queries/local/exception2.sql    |   10 -
 hplsql/src/test/queries/local/exception3.sql    |    5 -
 hplsql/src/test/queries/local/exception4.sql    |    7 -
 hplsql/src/test/queries/local/exception5.sql    |   10 -
 .../db/create_procedure_return_cursor.out.txt   |  135 +
 .../db/create_procedure_return_cursor2.out.txt  |  139 +
 .../test/results/db/cursor_attributes.out.txt   |   33 +
 hplsql/src/test/results/db/select_into.out.txt  |   19 +
 .../test/results/db/set_current_schema.out.txt  |   12 +
 .../src/test/results/db/sys_refcursor.out.txt   |   36 +
 hplsql/src/test/results/db/use.out.txt          |    4 +
 .../test/resources/testconfiguration.properties |    5 +
 .../org/apache/hive/jdbc/HiveConnection.java    |    4 +
 .../hadoop/hive/metastore/HiveMetaStore.java    |   11 +-
 .../hadoop/hive/metastore/txn/TxnHandler.java   |    9 +-
 .../metastore/txn/ValidCompactorTxnList.java    |    2 +-
 pom.xml                                         |    1 +
 ql/pom.xml                                      |    1 +
 .../UDAFTemplates/VectorUDAFMinMaxString.txt    |    3 +-
 .../java/org/apache/hadoop/hive/ql/Context.java |    1 -
 .../java/org/apache/hadoop/hive/ql/Driver.java  |  202 +-
 .../org/apache/hadoop/hive/ql/ErrorMsg.java     |    2 +-
 .../org/apache/hadoop/hive/ql/QueryPlan.java    |   18 +-
 .../apache/hadoop/hive/ql/exec/ExplainTask.java |    7 +-
 .../apache/hadoop/hive/ql/exec/JoinUtil.java    |   87 +-
 .../hadoop/hive/ql/exec/MapJoinOperator.java    |    2 +-
 .../apache/hadoop/hive/ql/exec/MoveTask.java    |    2 +-
 .../apache/hadoop/hive/ql/exec/Operator.java    |    6 +
 .../apache/hadoop/hive/ql/exec/Utilities.java   |    9 +-
 .../hive/ql/exec/tez/KeyValuesAdapter.java      |   47 +
 .../hive/ql/exec/tez/KeyValuesFromKeyValue.java |   90 +
 .../ql/exec/tez/KeyValuesFromKeyValues.java     |   48 +
 .../hive/ql/exec/tez/ReduceRecordProcessor.java |   11 +-
 .../hive/ql/exec/tez/ReduceRecordSource.java    |   12 +-
 .../apache/hadoop/hive/ql/exec/tez/TezTask.java |   10 +-
 .../hive/ql/exec/vector/BytesColumnVector.java  |  347 ---
 .../hive/ql/exec/vector/ColumnVector.java       |  178 --
 .../ql/exec/vector/DecimalColumnVector.java     |  125 -
 .../hive/ql/exec/vector/DoubleColumnVector.java |  161 -
 .../hive/ql/exec/vector/LongColumnVector.java   |  205 --
 .../ql/exec/vector/VectorMapJoinOperator.java   |    1 -
 .../hive/ql/exec/vector/VectorizedRowBatch.java |  186 --
 .../mapjoin/VectorMapJoinCommonOperator.java    |    1 +
 .../hadoop/hive/ql/io/orc/MemoryManager.java    |    4 +-
 .../apache/hadoop/hive/ql/io/orc/OrcConf.java   |  191 ++
 .../apache/hadoop/hive/ql/io/orc/OrcFile.java   |  158 +-
 .../hadoop/hive/ql/io/orc/OrcInputFormat.java   |    4 +-
 .../hadoop/hive/ql/io/orc/OrcOutputFormat.java  |   67 +-
 .../hive/ql/io/orc/OrcRawRecordMerger.java      |   20 +-
 .../apache/hadoop/hive/ql/io/orc/Reader.java    |   34 +-
 .../hadoop/hive/ql/io/orc/RecordReaderImpl.java |   63 +-
 .../hive/ql/io/orc/TreeReaderFactory.java       |   12 +-
 .../hadoop/hive/ql/io/orc/WriterImpl.java       |   27 +-
 .../read/ParquetRecordReaderWrapper.java        |    7 +-
 .../hive/ql/io/sarg/ConvertAstToSearchArg.java  |  439 +++
 .../hive/ql/io/sarg/SearchArgumentFactory.java  |   56 -
 .../hive/ql/io/sarg/SearchArgumentImpl.java     | 1027 -------
 .../apache/hadoop/hive/ql/lib/RuleRegExp.java   |  191 +-
 .../hadoop/hive/ql/lockmgr/DbTxnManager.java    |   36 +-
 .../hadoop/hive/ql/lockmgr/DummyTxnManager.java |    8 +
 .../hadoop/hive/ql/lockmgr/HiveTxnManager.java  |   21 +
 .../hive/ql/lockmgr/HiveTxnManagerImpl.java     |   10 +
 .../hadoop/hive/ql/lockmgr/LockException.java   |    8 +-
 .../hadoop/hive/ql/metadata/HiveException.java  |    3 +
 .../hadoop/hive/ql/optimizer/ColumnPruner.java  |    4 +
 .../hive/ql/optimizer/ColumnPrunerProcCtx.java  |   95 +-
 .../ql/optimizer/ColumnPrunerProcFactory.java   |   42 +-
 .../optimizer/ConstantPropagateProcFactory.java |  100 +-
 .../hive/ql/optimizer/ConvertJoinMapJoin.java   |  213 +-
 .../hive/ql/optimizer/GenMapRedUtils.java       |   18 +-
 .../hive/ql/optimizer/MapJoinProcessor.java     |   44 +-
 .../ql/optimizer/ReduceSinkMapJoinProc.java     |   84 +-
 .../hive/ql/optimizer/SimpleFetchOptimizer.java |    5 +-
 .../calcite/rules/HiveJoinToMultiJoinRule.java  |   82 +-
 .../calcite/translator/HiveOpConverter.java     |    2 +-
 .../calcite/translator/JoinTypeCheckCtx.java    |    2 +-
 .../translator/PlanModifierForASTConv.java      |    2 +-
 .../translator/PlanModifierForReturnPath.java   |    6 +-
 .../stats/annotation/StatsRulesProcFactory.java |    3 +
 .../hive/ql/parse/BaseSemanticAnalyzer.java     |   13 +
 .../hadoop/hive/ql/parse/CalcitePlanner.java    |    3 +-
 .../hive/ql/parse/ExplainSemanticAnalyzer.java  |    2 +-
 .../hadoop/hive/ql/parse/GenTezProcContext.java |   12 +
 .../hadoop/hive/ql/parse/GenTezUtils.java       |   23 +-
 .../apache/hadoop/hive/ql/parse/GenTezWork.java |   81 +-
 .../org/apache/hadoop/hive/ql/parse/HiveLexer.g |   11 +
 .../apache/hadoop/hive/ql/parse/HiveParser.g    |   70 +
 .../hadoop/hive/ql/parse/IdentifiersParser.g    |   19 +-
 .../apache/hadoop/hive/ql/parse/ParseUtils.java |   51 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |   61 +-
 .../hive/ql/parse/SemanticAnalyzerFactory.java  |   12 +
 .../hadoop/hive/ql/parse/TypeCheckCtx.java      |   15 +-
 .../hive/ql/parse/TypeCheckProcFactory.java     |    4 +
 .../apache/hadoop/hive/ql/plan/BaseWork.java    |    2 +-
 .../hive/ql/plan/CommonMergeJoinDesc.java       |    4 +
 .../apache/hadoop/hive/ql/plan/ExplainWork.java |   19 +-
 .../hadoop/hive/ql/plan/ExprNodeDescUtils.java  |  115 +
 .../hadoop/hive/ql/plan/HiveOperation.java      |   32 +-
 .../apache/hadoop/hive/ql/plan/MapJoinDesc.java |   11 +
 .../hadoop/hive/ql/plan/PartitionDesc.java      |   39 +-
 .../apache/hadoop/hive/ql/plan/ReduceWork.java  |    2 +-
 .../ql/processors/CommandProcessorResponse.java |   21 +-
 .../hadoop/hive/ql/processors/HiveCommand.java  |    3 +
 .../authorization/plugin/HiveOperationType.java |    5 +
 .../plugin/sqlstd/Operation2Privilege.java      |   11 +
 .../hadoop/hive/ql/session/SessionState.java    |   34 +-
 .../apache/hadoop/hive/ql/stats/StatsUtils.java |   10 +-
 .../hive/ql/udf/generic/GenericUDFBridge.java   |    8 +-
 .../apache/hadoop/hive/ql/TestTxnCommands.java  |  473 +++
 .../apache/hadoop/hive/ql/TestTxnCommands2.java |   68 +-
 .../exec/vector/TestVectorizedRowBatchCtx.java  |    6 +-
 .../hive/ql/io/orc/TestInputOutputFormat.java   |   29 +-
 .../hadoop/hive/ql/io/orc/TestOrcFile.java      |   11 +-
 .../hive/ql/io/orc/TestRecordReaderImpl.java    |   63 +-
 .../hive/ql/io/orc/TestVectorizedORCReader.java |   75 +-
 .../parquet/TestParquetRecordReaderWrapper.java |  155 +
 .../ql/io/sarg/TestConvertAstToSearchArg.java   | 2863 +++++++++++++++++
 .../hive/ql/io/sarg/TestSearchArgumentImpl.java | 2891 +-----------------
 .../hadoop/hive/ql/lib/TestRuleRegExp.java      |  118 +
 .../parse/TestUpdateDeleteSemanticAnalyzer.java |   34 +-
 .../positive/TestTransactionStatement.java      |  102 +
 .../hive/ql/session/TestSessionState.java       |    2 +-
 .../test/queries/clientpositive/stats_ppr_all.q |   24 +
 .../clientpositive/tez_dynpart_hashjoin_1.q     |  101 +
 .../clientpositive/tez_dynpart_hashjoin_2.q     |   83 +
 .../tez_vector_dynpart_hashjoin_1.q             |  102 +
 .../tez_vector_dynpart_hashjoin_2.q             |   84 +
 .../clientpositive/udf_from_utc_timestamp.q     |   30 +-
 .../clientpositive/udf_to_utc_timestamp.q       |   30 +-
 .../clientpositive/unionall_unbalancedppd.q     |  120 +
 .../test/queries/clientpositive/vector_acid3.q  |   17 +
 .../vector_aggregate_without_gby.q              |   14 +
 .../clientnegative/exchange_partition.q.out     |    2 +-
 .../clientpositive/exchange_partition.q.out     |    4 +-
 .../clientpositive/exchange_partition2.q.out    |    4 +-
 .../clientpositive/exchange_partition3.q.out    |    4 +-
 .../results/clientpositive/spark/union16.q.out  |   16 +-
 .../results/clientpositive/spark/union2.q.out   |   16 +-
 .../results/clientpositive/spark/union9.q.out   |   16 +-
 .../clientpositive/spark/union_view.q.out       |   24 +
 .../results/clientpositive/stats_ppr_all.q.out  |  284 ++
 .../tez/tez_dynpart_hashjoin_1.q.out            |  791 +++++
 .../tez/tez_dynpart_hashjoin_2.q.out            |  564 ++++
 .../tez/tez_vector_dynpart_hashjoin_1.q.out     |  804 +++++
 .../tez/tez_vector_dynpart_hashjoin_2.q.out     |  570 ++++
 .../results/clientpositive/tez/union2.q.out     |   28 +-
 .../results/clientpositive/tez/union9.q.out     |   40 +-
 .../clientpositive/tez/vector_acid3.q.out       |   31 +
 .../tez/vector_null_projection.q.out            |    4 +
 .../clientpositive/udf_from_utc_timestamp.q.out |   66 +-
 .../clientpositive/udf_to_utc_timestamp.q.out   |   66 +-
 .../test/results/clientpositive/union16.q.out   |  354 ++-
 ql/src/test/results/clientpositive/union2.q.out |   32 +-
 ql/src/test/results/clientpositive/union9.q.out |   46 +-
 .../results/clientpositive/union_view.q.out     |   24 +
 .../clientpositive/unionall_unbalancedppd.q.out |  373 +++
 .../results/clientpositive/vector_acid3.q.out   |   31 +
 .../vector_aggregate_without_gby.q.out          |   96 +
 .../clientpositive/vector_null_projection.q.out |    4 +
 .../hadoop/hive/ql/io/sarg/ExpressionTree.java  |  157 -
 .../hadoop/hive/ql/io/sarg/PredicateLeaf.java   |   87 -
 .../hadoop/hive/ql/io/sarg/SearchArgument.java  |  278 --
 .../apache/hadoop/hive/serde2/SerDeUtils.java   |   14 +-
 .../hadoop/hive/serde2/avro/InstanceCache.java  |    9 +-
 .../hive/serde2/io/HiveDecimalWritable.java     |  185 --
 .../hive/serde2/lazy/LazyHiveDecimal.java       |    3 +-
 .../lazy/fast/LazySimpleDeserializeRead.java    |   16 +-
 .../lazybinary/LazyBinaryHiveDecimal.java       |    2 +-
 .../hive/serde2/lazybinary/LazyBinarySerDe.java |   26 +-
 .../fast/LazyBinaryDeserializeRead.java         |    6 +-
 .../fast/LazyBinarySerializeWrite.java          |    5 +-
 .../hive/serde2/typeinfo/HiveDecimalUtils.java  |   35 +-
 .../hive/serde2/avro/TestInstanceCache.java     |   40 +-
 storage-api/pom.xml                             |   85 +
 .../hadoop/hive/common/type/HiveDecimal.java    |  312 ++
 .../hive/ql/exec/vector/BytesColumnVector.java  |  322 ++
 .../hive/ql/exec/vector/ColumnVector.java       |  173 ++
 .../ql/exec/vector/DecimalColumnVector.java     |  106 +
 .../hive/ql/exec/vector/DoubleColumnVector.java |  143 +
 .../hive/ql/exec/vector/LongColumnVector.java   |  189 ++
 .../hive/ql/exec/vector/VectorizedRowBatch.java |  186 ++
 .../hadoop/hive/ql/io/sarg/ExpressionTree.java  |  156 +
 .../hadoop/hive/ql/io/sarg/PredicateLeaf.java   |  104 +
 .../hadoop/hive/ql/io/sarg/SearchArgument.java  |  287 ++
 .../hive/ql/io/sarg/SearchArgumentFactory.java  |   28 +
 .../hive/ql/io/sarg/SearchArgumentImpl.java     |  687 +++++
 .../hive/serde2/io/HiveDecimalWritable.java     |  174 ++
 227 files changed, 16038 insertions(+), 7381 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/714b3db6/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/714b3db6/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/714b3db6/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/714b3db6/ql/pom.xml
----------------------------------------------------------------------
diff --cc ql/pom.xml
index 9bd7bba,6026c49..d8451c5
--- a/ql/pom.xml
+++ b/ql/pom.xml
@@@ -721,6 -716,6 +721,7 @@@
                    <include>com.google.guava:guava</include>
                    <include>net.sf.opencsv:opencsv</include>
                    <include>org.apache.hive:spark-client</include>
++                  <include>org.apache.hive:hive-storage-api</include>
                    <include>joda-time:joda-time</include>
                  </includes>
                </artifactSet>

http://git-wip-us.apache.org/repos/asf/hive/blob/714b3db6/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/714b3db6/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/714b3db6/ql/src/test/results/clientpositive/spark/union16.q.out
----------------------------------------------------------------------
diff --cc ql/src/test/results/clientpositive/spark/union16.q.out
index 39ba7b9,6e45714..5170bbd
--- a/ql/src/test/results/clientpositive/spark/union16.q.out
+++ b/ql/src/test/results/clientpositive/spark/union16.q.out
@@@ -78,12 -78,494 +78,14 @@@ STAGE PLANS
          Map 1 
              Map Operator Tree:
                  TableScan
 -                  alias: src
 -                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                  Select Operator
 -                    expressions: value (type: string)
 -                    outputColumnNames: _col0
 -                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                    Select Operator
 -                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
 -                      Group By Operator
 -                        aggregations: count(1)
 -                        mode: hash
 -                        outputColumnNames: _col0
 -                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                        Reduce Output Operator
 -                          sort order: 
 -                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                          value expressions: _col0 (type: bigint)
 -        Map 10 
 -            Map Operator Tree:
 -                TableScan
 -                  alias: src
 -                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                  Select Operator
 -                    expressions: value (type: string)
 -                    outputColumnNames: _col0
 -                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                    Select Operator
 -                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
 -                      Group By Operator
 -                        aggregations: count(1)
 -                        mode: hash
 -                        outputColumnNames: _col0
 -                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                        Reduce Output Operator
 -                          sort order: 
 -                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                          value expressions: _col0 (type: bigint)
 -        Map 11 
 -            Map Operator Tree:
 -                TableScan
 -                  alias: src
 -                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                  Select Operator
 -                    expressions: value (type: string)
 -                    outputColumnNames: _col0
 -                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                    Select Operator
 -                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
 -                      Group By Operator
 -                        aggregations: count(1)
 -                        mode: hash
 -                        outputColumnNames: _col0
 -                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                        Reduce Output Operator
 -                          sort order: 
 -                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                          value expressions: _col0 (type: bigint)
 -        Map 12 
 -            Map Operator Tree:
 -                TableScan
 -                  alias: src
 -                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                  Select Operator
 -                    expressions: value (type: string)
 -                    outputColumnNames: _col0
 -                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                    Select Operator
 -                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
 -                      Group By Operator
 -                        aggregations: count(1)
 -                        mode: hash
 -                        outputColumnNames: _col0
 -                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                        Reduce Output Operator
 -                          sort order: 
 -                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                          value expressions: _col0 (type: bigint)
 -        Map 13 
 -            Map Operator Tree:
 -                TableScan
 -                  alias: src
 -                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                  Select Operator
 -                    expressions: value (type: string)
 -                    outputColumnNames: _col0
 -                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                    Select Operator
 -                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
 -                      Group By Operator
 -                        aggregations: count(1)
 -                        mode: hash
 -                        outputColumnNames: _col0
 -                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                        Reduce Output Operator
 -                          sort order: 
 -                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                          value expressions: _col0 (type: bigint)
 -        Map 14 
 -            Map Operator Tree:
 -                TableScan
 -                  alias: src
 -                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                  Select Operator
 -                    expressions: value (type: string)
 -                    outputColumnNames: _col0
 -                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                    Select Operator
 -                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
 -                      Group By Operator
 -                        aggregations: count(1)
 -                        mode: hash
 -                        outputColumnNames: _col0
 -                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                        Reduce Output Operator
 -                          sort order: 
 -                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                          value expressions: _col0 (type: bigint)
 -        Map 15 
 -            Map Operator Tree:
 -                TableScan
 -                  alias: src
 -                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                  Select Operator
 -                    expressions: value (type: string)
 -                    outputColumnNames: _col0
 -                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                    Select Operator
 -                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
 -                      Group By Operator
 -                        aggregations: count(1)
 -                        mode: hash
 -                        outputColumnNames: _col0
 -                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                        Reduce Output Operator
 -                          sort order: 
 -                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                          value expressions: _col0 (type: bigint)
 -        Map 16 
 -            Map Operator Tree:
 -                TableScan
 -                  alias: src
 -                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                  Select Operator
 -                    expressions: value (type: string)
 -                    outputColumnNames: _col0
 -                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                    Select Operator
 -                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
 -                      Group By Operator
 -                        aggregations: count(1)
 -                        mode: hash
 -                        outputColumnNames: _col0
 -                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                        Reduce Output Operator
 -                          sort order: 
 -                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                          value expressions: _col0 (type: bigint)
 -        Map 17 
 -            Map Operator Tree:
 -                TableScan
 -                  alias: src
 -                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                  Select Operator
 -                    expressions: value (type: string)
 -                    outputColumnNames: _col0
 -                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                    Select Operator
 -                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
 -                      Group By Operator
 -                        aggregations: count(1)
 -                        mode: hash
 -                        outputColumnNames: _col0
 -                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                        Reduce Output Operator
 -                          sort order: 
 -                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                          value expressions: _col0 (type: bigint)
 -        Map 18 
 -            Map Operator Tree:
 -                TableScan
 -                  alias: src
 -                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                  Select Operator
 -                    expressions: value (type: string)
 -                    outputColumnNames: _col0
 -                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                    Select Operator
 -                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
 -                      Group By Operator
 -                        aggregations: count(1)
 -                        mode: hash
 -                        outputColumnNames: _col0
 -                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                        Reduce Output Operator
 -                          sort order: 
 -                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                          value expressions: _col0 (type: bigint)
 -        Map 19 
 -            Map Operator Tree:
 -                TableScan
 -                  alias: src
 -                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                  Select Operator
 -                    expressions: value (type: string)
 -                    outputColumnNames: _col0
 -                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                    Select Operator
 -                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
 -                      Group By Operator
 -                        aggregations: count(1)
 -                        mode: hash
 -                        outputColumnNames: _col0
 -                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                        Reduce Output Operator
 -                          sort order: 
 -                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                          value expressions: _col0 (type: bigint)
 -        Map 20 
 -            Map Operator Tree:
 -                TableScan
 -                  alias: src
 -                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                  Select Operator
 -                    expressions: value (type: string)
 -                    outputColumnNames: _col0
 -                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                    Select Operator
 -                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
 -                      Group By Operator
 -                        aggregations: count(1)
 -                        mode: hash
 -                        outputColumnNames: _col0
 -                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                        Reduce Output Operator
 -                          sort order: 
 -                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                          value expressions: _col0 (type: bigint)
 -        Map 21 
 -            Map Operator Tree:
 -                TableScan
 -                  alias: src
 -                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                  Select Operator
 -                    expressions: value (type: string)
 -                    outputColumnNames: _col0
 -                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                    Select Operator
 -                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
 -                      Group By Operator
 -                        aggregations: count(1)
 -                        mode: hash
 -                        outputColumnNames: _col0
 -                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                        Reduce Output Operator
 -                          sort order: 
 -                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                          value expressions: _col0 (type: bigint)
 -        Map 22 
 -            Map Operator Tree:
 -                TableScan
 -                  alias: src
 -                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                  Select Operator
 -                    expressions: value (type: string)
 -                    outputColumnNames: _col0
 -                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                    Select Operator
 -                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
 -                      Group By Operator
 -                        aggregations: count(1)
 -                        mode: hash
 -                        outputColumnNames: _col0
 -                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                        Reduce Output Operator
 -                          sort order: 
 -                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                          value expressions: _col0 (type: bigint)
 -        Map 23 
 -            Map Operator Tree:
 -                TableScan
 -                  alias: src
 -                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                  Select Operator
 -                    expressions: value (type: string)
 -                    outputColumnNames: _col0
 -                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                    Select Operator
 -                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
 -                      Group By Operator
 -                        aggregations: count(1)
 -                        mode: hash
 -                        outputColumnNames: _col0
 -                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                        Reduce Output Operator
 -                          sort order: 
 -                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                          value expressions: _col0 (type: bigint)
 -        Map 24 
 -            Map Operator Tree:
 -                TableScan
 -                  alias: src
 -                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                  Select Operator
 -                    expressions: value (type: string)
 -                    outputColumnNames: _col0
 -                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                    Select Operator
 -                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
 -                      Group By Operator
 -                        aggregations: count(1)
 -                        mode: hash
 -                        outputColumnNames: _col0
 -                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                        Reduce Output Operator
 -                          sort order: 
 -                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                          value expressions: _col0 (type: bigint)
 -        Map 25 
 -            Map Operator Tree:
 -                TableScan
 -                  alias: src
 -                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                  Select Operator
 -                    expressions: value (type: string)
 -                    outputColumnNames: _col0
 -                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                    Select Operator
 -                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
 -                      Group By Operator
 -                        aggregations: count(1)
 -                        mode: hash
 -                        outputColumnNames: _col0
 -                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                        Reduce Output Operator
 -                          sort order: 
 -                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                          value expressions: _col0 (type: bigint)
 -        Map 26 
 -            Map Operator Tree:
 -                TableScan
 -                  alias: src
 -                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                  Select Operator
 -                    expressions: value (type: string)
 -                    outputColumnNames: _col0
 -                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                    Select Operator
 -                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
 -                      Group By Operator
 -                        aggregations: count(1)
 -                        mode: hash
 -                        outputColumnNames: _col0
 -                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                        Reduce Output Operator
 -                          sort order: 
 -                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                          value expressions: _col0 (type: bigint)
 -        Map 3 
 -            Map Operator Tree:
 -                TableScan
 -                  alias: src
 -                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                  Select Operator
 -                    expressions: value (type: string)
 -                    outputColumnNames: _col0
 -                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                    Select Operator
 -                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
 -                      Group By Operator
 -                        aggregations: count(1)
 -                        mode: hash
 -                        outputColumnNames: _col0
 -                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                        Reduce Output Operator
 -                          sort order: 
 -                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                          value expressions: _col0 (type: bigint)
 -        Map 4 
 -            Map Operator Tree:
 -                TableScan
 -                  alias: src
 -                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                  Select Operator
 -                    expressions: value (type: string)
 -                    outputColumnNames: _col0
 -                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                    Select Operator
 -                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
 -                      Group By Operator
 -                        aggregations: count(1)
 -                        mode: hash
 -                        outputColumnNames: _col0
 -                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                        Reduce Output Operator
 -                          sort order: 
 -                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                          value expressions: _col0 (type: bigint)
 -        Map 5 
 -            Map Operator Tree:
 -                TableScan
 -                  alias: src
 -                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                  Select Operator
 -                    expressions: value (type: string)
 -                    outputColumnNames: _col0
 -                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                    Select Operator
 -                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
 -                      Group By Operator
 -                        aggregations: count(1)
 -                        mode: hash
 -                        outputColumnNames: _col0
 -                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                        Reduce Output Operator
 -                          sort order: 
 -                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                          value expressions: _col0 (type: bigint)
 -        Map 6 
 -            Map Operator Tree:
 -                TableScan
 -                  alias: src
 -                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                  Select Operator
 -                    expressions: value (type: string)
 -                    outputColumnNames: _col0
 -                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                    Select Operator
 -                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
 -                      Group By Operator
 -                        aggregations: count(1)
 -                        mode: hash
 -                        outputColumnNames: _col0
 -                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                        Reduce Output Operator
 -                          sort order: 
 -                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                          value expressions: _col0 (type: bigint)
 -        Map 7 
 -            Map Operator Tree:
 -                TableScan
 -                  alias: src
 -                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                  Select Operator
 -                    expressions: value (type: string)
 -                    outputColumnNames: _col0
 -                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                    Select Operator
 -                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
 -                      Group By Operator
 -                        aggregations: count(1)
 -                        mode: hash
 -                        outputColumnNames: _col0
 -                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                        Reduce Output Operator
 -                          sort order: 
 -                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                          value expressions: _col0 (type: bigint)
 -        Map 8 
 -            Map Operator Tree:
 -                TableScan
 -                  alias: src
 -                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                  Select Operator
 -                    expressions: value (type: string)
 -                    outputColumnNames: _col0
 -                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                    Select Operator
 -                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
 -                      Group By Operator
 -                        aggregations: count(1)
 -                        mode: hash
 -                        outputColumnNames: _col0
 -                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                        Reduce Output Operator
 -                          sort order: 
 -                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                          value expressions: _col0 (type: bigint)
 -        Map 9 
 -            Map Operator Tree:
 -                TableScan
                    alias: src
-                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                    Select Operator
-                     Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                     expressions: value (type: string)
+                     outputColumnNames: _col0
+                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                      Select Operator
-                       Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                       Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                        Group By Operator
                          aggregations: count(1)
                          mode: hash

http://git-wip-us.apache.org/repos/asf/hive/blob/714b3db6/ql/src/test/results/clientpositive/spark/union2.q.out
----------------------------------------------------------------------
diff --cc ql/src/test/results/clientpositive/spark/union2.q.out
index 3c5b075,9e44bef..8b516b5
--- a/ql/src/test/results/clientpositive/spark/union2.q.out
+++ b/ql/src/test/results/clientpositive/spark/union2.q.out
@@@ -27,11 -27,33 +27,13 @@@ STAGE PLANS
              Map Operator Tree:
                  TableScan
                    alias: s1
-                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                    Select Operator
-                     Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                     expressions: value (type: string)
+                     outputColumnNames: _col0
+                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                      Select Operator
-                       Statistics: Num rows: 1000 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
 -                      Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
 -                      Group By Operator
 -                        aggregations: count(1)
 -                        mode: hash
 -                        outputColumnNames: _col0
 -                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                        Reduce Output Operator
 -                          sort order: 
 -                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                          value expressions: _col0 (type: bigint)
 -        Map 3 
 -            Map Operator Tree:
 -                TableScan
 -                  alias: s1
 -                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                  Select Operator
 -                    expressions: value (type: string)
 -                    outputColumnNames: _col0
 -                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                    Select Operator
+                       Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
                        Group By Operator
                          aggregations: count(1)
                          mode: hash

http://git-wip-us.apache.org/repos/asf/hive/blob/714b3db6/ql/src/test/results/clientpositive/spark/union9.q.out
----------------------------------------------------------------------
diff --cc ql/src/test/results/clientpositive/spark/union9.q.out
index 92499c0,cb41d91..44c5f6b
--- a/ql/src/test/results/clientpositive/spark/union9.q.out
+++ b/ql/src/test/results/clientpositive/spark/union9.q.out
@@@ -29,19 -29,61 +29,21 @@@ STAGE PLANS
              Map Operator Tree:
                  TableScan
                    alias: s1
-                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                    Select Operator
-                     Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                     expressions: value (type: string)
+                     outputColumnNames: _col0
+                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                      Select Operator
-                       Statistics: Num rows: 1500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                       Statistics: Num rows: 1500 Data size: 15936 Basic stats: COMPLETE Column stats: NONE
                        Group By Operator
                          aggregations: count(1)
 -                        mode: hash
 -                        outputColumnNames: _col0
 -                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                        Reduce Output Operator
 -                          sort order: 
 -                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                          value expressions: _col0 (type: bigint)
 -        Map 3 
 -            Map Operator Tree:
 -                TableScan
 -                  alias: s1
 -                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                  Select Operator
 -                    expressions: value (type: string)
 -                    outputColumnNames: _col0
 -                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                    Select Operator
 -                      Statistics: Num rows: 1500 Data size: 15936 Basic stats: COMPLETE Column stats: NONE
 -                      Group By Operator
 -                        aggregations: count(1)
 -                        mode: hash
 -                        outputColumnNames: _col0
 -                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                        Reduce Output Operator
 -                          sort order: 
 -                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 -                          value expressions: _col0 (type: bigint)
 -        Map 4 
 -            Map Operator Tree:
 -                TableScan
 -                  alias: s1
 -                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                  Select Operator
 -                    expressions: value (type: string)
 -                    outputColumnNames: _col0
 -                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
 -                    Select Operator
 -                      Statistics: Num rows: 1500 Data size: 15936 Basic stats: COMPLETE Column stats: NONE
 -                      Group By Operator
 -                        aggregations: count(1)
                          mode: hash
                          outputColumnNames: _col0
-                         Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                         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: COMPLETE
+                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                            value expressions: _col0 (type: bigint)
          Reducer 2 
              Reduce Operator Tree:

http://git-wip-us.apache.org/repos/asf/hive/blob/714b3db6/ql/src/test/results/clientpositive/spark/union_view.q.out
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/714b3db6/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleDeserializeRead.java
----------------------------------------------------------------------
diff --cc serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleDeserializeRead.java
index 8c5b0b3,7588106..dc76c7d
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleDeserializeRead.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleDeserializeRead.java
@@@ -30,16 -30,15 +30,13 @@@ import org.apache.hadoop.hive.common.ty
  import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
  import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth;
  import org.apache.hadoop.hive.serde2.fast.DeserializeRead;
- import org.apache.hadoop.hive.serde2.fast.DeserializeRead.ReadIntervalDayTimeResults;
- import org.apache.hadoop.hive.serde2.fast.DeserializeRead.ReadIntervalYearMonthResults;
+ 
  import org.apache.hadoop.hive.serde2.io.DateWritable;
  import org.apache.hadoop.hive.serde2.io.HiveCharWritable;
--import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
  import org.apache.hadoop.hive.serde2.io.HiveIntervalDayTimeWritable;
  import org.apache.hadoop.hive.serde2.io.HiveIntervalYearMonthWritable;
  import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable;
  import org.apache.hadoop.hive.serde2.io.TimestampWritable;
--import org.apache.hadoop.hive.serde2.lazy.ByteArrayRef;
  import org.apache.hadoop.hive.serde2.lazy.LazyBinary;
  import org.apache.hadoop.hive.serde2.lazy.LazyByte;
  import org.apache.hadoop.hive.serde2.lazy.LazyInteger;
@@@ -47,14 -46,8 +44,8 @@@ import org.apache.hadoop.hive.serde2.la
  import org.apache.hadoop.hive.serde2.lazy.LazySerDeParameters;
  import org.apache.hadoop.hive.serde2.lazy.LazyShort;
  import org.apache.hadoop.hive.serde2.lazy.LazyUtils;
- import org.apache.hadoop.hive.serde2.lazy.objectinspector.LazySimpleStructObjectInspector;
- import org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive.LazyObjectInspectorParameters;
- import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
- import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
  import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo;
 -import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;;
 +import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
- import org.apache.hadoop.hive.serde2.typeinfo.HiveDecimalUtils;
  import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
  import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo;
  import org.apache.hadoop.io.Text;


[32/43] hive git commit: HIVE-11307. Remove getWritableObject from ColumnVectorBatch. (omalley reviewed by prasanthj)

Posted by xu...@apache.org.
HIVE-11307. Remove getWritableObject from ColumnVectorBatch. (omalley
reviewed by prasanthj)


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

Branch: refs/heads/spark
Commit: 0ead9fe6895d582ccbd289a4576b34f7d2ca15b8
Parents: 29651cd
Author: Owen O'Malley <om...@apache.org>
Authored: Tue Jul 28 12:57:39 2015 -0700
Committer: Owen O'Malley <om...@apache.org>
Committed: Tue Jul 28 12:57:39 2015 -0700

----------------------------------------------------------------------
 .../hive/ql/exec/vector/BytesColumnVector.java  | 27 +------
 .../hive/ql/exec/vector/ColumnVector.java       |  4 --
 .../ql/exec/vector/DecimalColumnVector.java     | 23 +-----
 .../hive/ql/exec/vector/DoubleColumnVector.java | 18 -----
 .../hive/ql/exec/vector/LongColumnVector.java   | 24 ++-----
 .../exec/vector/TestVectorizedRowBatchCtx.java  |  6 +-
 .../hive/ql/io/orc/TestInputOutputFormat.java   |  5 +-
 .../hive/ql/io/orc/TestVectorizedORCReader.java | 75 ++++++++++++++------
 8 files changed, 68 insertions(+), 114 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/0ead9fe6/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/BytesColumnVector.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/BytesColumnVector.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/BytesColumnVector.java
index c9a0fa2..02c52fa 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/BytesColumnVector.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/BytesColumnVector.java
@@ -18,12 +18,6 @@
 
 package org.apache.hadoop.hive.ql.exec.vector;
 
-import java.util.Arrays;
-
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-
 /**
  * This class supports string and binary data by value reference -- i.e. each field is
  * explicitly present, as opposed to provided by a dictionary reference.
@@ -51,9 +45,6 @@ public class BytesColumnVector extends ColumnVector {
   private byte[] buffer;   // optional buffer to use when actually copying in data
   private int nextFree;    // next free position in buffer
 
-  // Reusable text object
-  private final Text textObject = new Text();
-
   // Estimate that there will be 16 bytes per entry
   static final int DEFAULT_BUFFER_SIZE = 16 * VectorizedRowBatch.DEFAULT_SIZE;
 
@@ -215,22 +206,6 @@ public class BytesColumnVector extends ColumnVector {
     buffer = newBuffer;
   }
 
-  @Override
-  public Writable getWritableObject(int index) {
-    if (this.isRepeating) {
-      index = 0;
-    }
-    Writable result = null;
-    if (!isNull[index] && vector[index] != null) {
-      textObject.clear();
-      textObject.append(vector[index], start[index], length[index]);
-      result = textObject;
-    } else {
-      result = NullWritable.get();
-    }
-    return result;
-  }
-
   /** Copy the current object contents into the output. Only copy selected entries,
     * as indicated by selectedInUse and the sel array.
     */
@@ -294,7 +269,7 @@ public class BytesColumnVector extends ColumnVector {
 
       // Only copy data values if entry is not null. The string value
       // at position 0 is undefined if the position 0 value is null.
-      if (noNulls || (!noNulls && !isNull[0])) {
+      if (noNulls || !isNull[0]) {
 
         // loops start at position 1 because position 0 is already set
         if (selectedInUse) {

http://git-wip-us.apache.org/repos/asf/hive/blob/0ead9fe6/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ColumnVector.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ColumnVector.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ColumnVector.java
index 49d4c12..4b5cf39 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ColumnVector.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ColumnVector.java
@@ -20,8 +20,6 @@ package org.apache.hadoop.hive.ql.exec.vector;
 
 import java.util.Arrays;
 
-import org.apache.hadoop.io.Writable;
-
 /**
  * ColumnVector contains the shared structure for the sub-types,
  * including NULL information, and whether this vector
@@ -64,8 +62,6 @@ public abstract class ColumnVector {
   private boolean preFlattenIsRepeating;
   private boolean preFlattenNoNulls;
 
-  public abstract Writable getWritableObject(int index);
-
   /**
    * Constructor for super-class ColumnVector. This is not called directly,
    * but used to initialize inherited fields.

http://git-wip-us.apache.org/repos/asf/hive/blob/0ead9fe6/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/DecimalColumnVector.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/DecimalColumnVector.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/DecimalColumnVector.java
index 0f63b29..74a9d5f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/DecimalColumnVector.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/DecimalColumnVector.java
@@ -22,9 +22,6 @@ import java.math.BigInteger;
 
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
-import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.Writable;
 
 public class DecimalColumnVector extends ColumnVector {
 
@@ -39,8 +36,6 @@ public class DecimalColumnVector extends ColumnVector {
   public short scale;
   public short precision;
 
-  private final HiveDecimalWritable writableObj = new HiveDecimalWritable();
-
   public DecimalColumnVector(int precision, int scale) {
     this(VectorizedRowBatch.DEFAULT_SIZE, precision, scale);
   }
@@ -49,27 +44,13 @@ public class DecimalColumnVector extends ColumnVector {
     super(size);
     this.precision = (short) precision;
     this.scale = (short) scale;
-    final int len = size;
-    vector = new HiveDecimalWritable[len];
-    for (int i = 0; i < len; i++) {
+    vector = new HiveDecimalWritable[size];
+    for (int i = 0; i < size; i++) {
       vector[i] = new HiveDecimalWritable(HiveDecimal.ZERO);
     }
   }
 
   @Override
-  public Writable getWritableObject(int index) {
-    if (isRepeating) {
-      index = 0;
-    }
-    if (!noNulls && isNull[index]) {
-      return NullWritable.get();
-    } else {
-      writableObj.set(vector[index]);
-      return writableObj;
-    }
-  }
-
-  @Override
   public void flatten(boolean selectedInUse, int[] sel, int size) {
     // TODO Auto-generated method stub
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/0ead9fe6/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/DoubleColumnVector.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/DoubleColumnVector.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/DoubleColumnVector.java
index 013a9f5..4a7811d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/DoubleColumnVector.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/DoubleColumnVector.java
@@ -19,10 +19,6 @@ package org.apache.hadoop.hive.ql.exec.vector;
 
 import java.util.Arrays;
 
-import org.apache.hadoop.hive.serde2.io.DoubleWritable;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.Writable;
-
 /**
  * This class represents a nullable double precision floating point column vector.
  * This class will be used for operations on all floating point types (float, double)
@@ -36,7 +32,6 @@ import org.apache.hadoop.io.Writable;
  */
 public class DoubleColumnVector extends ColumnVector {
   public double[] vector;
-  private final DoubleWritable writableObj = new DoubleWritable();
   public static final double NULL_VALUE = Double.NaN;
 
   /**
@@ -57,19 +52,6 @@ public class DoubleColumnVector extends ColumnVector {
     vector = new double[len];
   }
 
-  @Override
-  public Writable getWritableObject(int index) {
-    if (this.isRepeating) {
-      index = 0;
-    }
-    if (!noNulls && isNull[index]) {
-      return NullWritable.get();
-    } else {
-      writableObj.set(vector[index]);
-      return writableObj;
-    }
-  }
-
   // Copy the current object contents into the output. Only copy selected entries,
   // as indicated by selectedInUse and the sel array.
   public void copySelected(

http://git-wip-us.apache.org/repos/asf/hive/blob/0ead9fe6/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/LongColumnVector.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/LongColumnVector.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/LongColumnVector.java
index d900cc6..5702584 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/LongColumnVector.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/LongColumnVector.java
@@ -19,10 +19,6 @@ package org.apache.hadoop.hive.ql.exec.vector;
 
 import java.util.Arrays;
 
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.Writable;
-
 /**
  * This class represents a nullable int column vector.
  * This class will be used for operations on all integer types (tinyint, smallint, int, bigint)
@@ -36,7 +32,6 @@ import org.apache.hadoop.io.Writable;
  */
 public class LongColumnVector extends ColumnVector {
   public long[] vector;
-  private final LongWritable writableObj = new LongWritable();
   public static final long NULL_VALUE = 1;
 
   /**
@@ -50,26 +45,13 @@ public class LongColumnVector extends ColumnVector {
   /**
    * Don't use this except for testing purposes.
    *
-   * @param len
+   * @param len the number of rows
    */
   public LongColumnVector(int len) {
     super(len);
     vector = new long[len];
   }
 
-  @Override
-  public Writable getWritableObject(int index) {
-    if (this.isRepeating) {
-      index = 0;
-    }
-    if (!noNulls && isNull[index]) {
-      return NullWritable.get();
-    } else {
-      writableObj.set(vector[index]);
-      return writableObj;
-    }
-  }
-
   // Copy the current object contents into the output. Only copy selected entries,
   // as indicated by selectedInUse and the sel array.
   public void copySelected(
@@ -141,7 +123,9 @@ public class LongColumnVector extends ColumnVector {
       }
     }
     else {
-      System.arraycopy(vector, 0, output.vector, 0, size);
+      for(int i = 0; i < size; ++i) {
+        output.vector[i] = vector[i];
+      }
     }
 
     // Copy nulls over if needed

http://git-wip-us.apache.org/repos/asf/hive/blob/0ead9fe6/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizedRowBatchCtx.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizedRowBatchCtx.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizedRowBatchCtx.java
index 473ebac..3321823 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizedRowBatchCtx.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizedRowBatchCtx.java
@@ -289,7 +289,8 @@ public class TestVectorizedRowBatchCtx {
           case BINARY: {
             BytesColumnVector bcv = (BytesColumnVector) batch.cols[j];
               BytesWritable colBinary = (BytesWritable) writableCol;
-              BytesWritable batchBinary = (BytesWritable) bcv.getWritableObject(i);
+              BytesWritable batchBinary = new BytesWritable();
+              batchBinary.set(bcv.vector[i], bcv.start[i], bcv.length[i]);
               byte[] a = colBinary.getBytes();
               byte[] b = batchBinary.getBytes();
               Assert.assertEquals(true, Arrays.equals(a, b));
@@ -298,7 +299,8 @@ public class TestVectorizedRowBatchCtx {
           case STRING: {
             BytesColumnVector bcv = (BytesColumnVector) batch.cols[j];
             Text colText = (Text) writableCol;
-            Text batchText = (Text) bcv.getWritableObject(i);
+            Text batchText = new Text();
+            batchText.set(bcv.vector[i], bcv.start[i], bcv.length[i]);
             String a = colText.toString();
             String b = batchText.toString();
             Assert.assertEquals(true, a.equals(b));

http://git-wip-us.apache.org/repos/asf/hive/blob/0ead9fe6/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
index e40e1d2..46deda5 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
@@ -1542,8 +1542,11 @@ public class TestInputOutputFormat {
       assertEquals("checking long " + i, i, longColumn.vector[i]);
       assertEquals("checking float " + i, i, floatColumn.vector[i], 0.0001);
       assertEquals("checking double " + i, i, doubleCoulmn.vector[i], 0.0001);
+      Text strValue = new Text();
+      strValue.set(stringColumn.vector[i], stringColumn.start[i],
+          stringColumn.length[i]);
       assertEquals("checking string " + i, new Text(Long.toHexString(i)),
-          stringColumn.getWritableObject(i));
+          strValue);
       assertEquals("checking decimal " + i, HiveDecimal.create(i),
           decimalColumn.vector[i].getHiveDecimal());
       assertEquals("checking date " + i, i, dateColumn.vector[i]);

http://git-wip-us.apache.org/repos/asf/hive/blob/0ead9fe6/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorizedORCReader.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorizedORCReader.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorizedORCReader.java
index e72e5cf..c739aef 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorizedORCReader.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorizedORCReader.java
@@ -30,18 +30,30 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.serde2.io.ByteWritable;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.serde2.io.DoubleWritable;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.serde2.io.ShortWritable;
 import org.apache.hadoop.hive.serde2.io.TimestampWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
 import org.apache.hadoop.io.BooleanWritable;
+import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
 import org.junit.Before;
 import org.junit.Test;
 
+import static org.junit.Assert.assertEquals;
+
 /**
 *
 * Class that tests ORC reader vectorization by comparing records that are
@@ -149,42 +161,61 @@ public class TestVectorizedORCReader {
         row = (OrcStruct) rr.next(row);
         for (int j = 0; j < batch.cols.length; j++) {
           Object a = (row.getFieldValue(j));
-          Object b = batch.cols[j].getWritableObject(i);
-          // Boolean values are stores a 1's and 0's, so convert and compare
-          if (a instanceof BooleanWritable) {
+          ColumnVector cv = batch.cols[j];
+          // if the value is repeating, use row 0
+          int rowId = cv.isRepeating ? 0 : i;
+
+          // make sure the null flag agrees
+          if (a == null) {
+            Assert.assertEquals(true, !cv.noNulls && cv.isNull[rowId]);
+          } else if (a instanceof BooleanWritable) {
+
+            // Boolean values are stores a 1's and 0's, so convert and compare
             Long temp = (long) (((BooleanWritable) a).get() ? 1 : 0);
-            Assert.assertEquals(true, temp.toString().equals(b.toString()));
-            continue;
-          }
-          // Timestamps are stored as long, so convert and compare
-          if (a instanceof TimestampWritable) {
+            long b = ((LongColumnVector) cv).vector[rowId];
+            Assert.assertEquals(temp.toString(), Long.toString(b));
+          } else if (a instanceof TimestampWritable) {
+            // Timestamps are stored as long, so convert and compare
             TimestampWritable t = ((TimestampWritable) a);
             // Timestamp.getTime() is overriden and is 
             // long time = super.getTime();
             // return (time + (nanos / 1000000));
             Long timeInNanoSec = (t.getTimestamp().getTime() * 1000000)
                 + (t.getTimestamp().getNanos() % 1000000);
-            Assert.assertEquals(true, timeInNanoSec.toString().equals(b.toString()));
-            continue;
-          }
+            long b = ((LongColumnVector) cv).vector[rowId];
+            Assert.assertEquals(timeInNanoSec.toString(), Long.toString(b));
+
+          } else if (a instanceof DateWritable) {
+            // Dates are stored as long, so convert and compare
 
-          // Dates are stored as long, so convert and compare
-          if (a instanceof DateWritable) {
             DateWritable adt = (DateWritable) a;
-            Assert.assertEquals(adt.get().getTime(), DateWritable.daysToMillis((int) ((LongWritable) b).get()));
-            continue;
-          }
+            long b = ((LongColumnVector) cv).vector[rowId];
+            Assert.assertEquals(adt.get().getTime(),
+                DateWritable.daysToMillis((int) b));
 
-          // Decimals are stored as BigInteger, so convert and compare
-          if (a instanceof HiveDecimalWritable) {
+          } else if (a instanceof HiveDecimalWritable) {
+            // Decimals are stored as BigInteger, so convert and compare
             HiveDecimalWritable dec = (HiveDecimalWritable) a;
+            HiveDecimalWritable b = ((DecimalColumnVector) cv).vector[i];
             Assert.assertEquals(dec, b);
-          }
 
-          if (null == a) {
-            Assert.assertEquals(true, (b == null || (b instanceof NullWritable)));
+          } else if (a instanceof DoubleWritable) {
+
+            double b = ((DoubleColumnVector) cv).vector[rowId];
+            assertEquals(a.toString(), Double.toString(b));
+          } else if (a instanceof Text) {
+            BytesColumnVector bcv = (BytesColumnVector) cv;
+            Text b = new Text();
+            b.set(bcv.vector[rowId], bcv.start[rowId], bcv.length[rowId]);
+            assertEquals(a, b);
+          } else if (a instanceof IntWritable ||
+              a instanceof LongWritable ||
+              a instanceof ByteWritable ||
+              a instanceof ShortWritable) {
+            assertEquals(a.toString(),
+                Long.toString(((LongColumnVector) cv).vector[rowId]));
           } else {
-            Assert.assertEquals(true, b.toString().equals(a.toString()));
+            assertEquals("huh", a.getClass().getName());
           }
         }
       }


[22/43] hive git commit: HIVE-11271 : java.lang.IndexOutOfBoundsException when union all with if function (Yongzhi Chen via Szehon)

Posted by xu...@apache.org.
HIVE-11271 : java.lang.IndexOutOfBoundsException when union all with if function (Yongzhi Chen via Szehon)


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

Branch: refs/heads/spark
Commit: c2dc452ece2d625a974a189d64cc42784597c5f6
Parents: 9f47426
Author: Szehon Ho <sz...@cloudera.com>
Authored: Mon Jul 27 11:34:12 2015 -0700
Committer: Szehon Ho <sz...@cloudera.com>
Committed: Mon Jul 27 11:34:12 2015 -0700

----------------------------------------------------------------------
 .../hive/ql/optimizer/ColumnPrunerProcCtx.java  |  64 ++++
 .../ql/optimizer/ColumnPrunerProcFactory.java   |   2 +-
 .../clientpositive/unionall_unbalancedppd.q     | 120 ++++++
 .../clientpositive/unionall_unbalancedppd.q.out | 373 +++++++++++++++++++
 4 files changed, 558 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/c2dc452e/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcCtx.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcCtx.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcCtx.java
index c076d4e..8bcb464 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcCtx.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcCtx.java
@@ -25,7 +25,9 @@ import java.util.Map;
 
 import org.apache.hadoop.hive.ql.exec.ColumnInfo;
 import org.apache.hadoop.hive.ql.exec.CommonJoinOperator;
+import org.apache.hadoop.hive.ql.exec.FilterOperator;
 import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.exec.OperatorFactory;
 import org.apache.hadoop.hive.ql.exec.RowSchema;
 import org.apache.hadoop.hive.ql.exec.SelectOperator;
 import org.apache.hadoop.hive.ql.exec.UnionOperator;
@@ -33,6 +35,7 @@ import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.OperatorDesc;
 import org.apache.hadoop.hive.ql.plan.SelectDesc;
@@ -241,4 +244,65 @@ public class ColumnPrunerProcCtx implements NodeProcessorCtx {
     }
     return columns;
   }
+
+  /**
+   * If the input filter operator has direct child(ren) which are union operator,
+   * and the filter's column is not the same as union's
+   * create select operator between them. The select operator has same number of columns as
+   * pruned child operator.
+   *
+   * @param curOp
+   *          The filter operator which need to handle children.
+   * @throws SemanticException
+   */
+  public void handleFilterUnionChildren(Operator<? extends OperatorDesc> curOp)
+      throws SemanticException {
+    if (curOp.getChildOperators() == null || !(curOp instanceof FilterOperator)) {
+      return;
+    }
+    List<String> parentPrunList = prunedColLists.get(curOp);
+    if(parentPrunList == null || parentPrunList.size() == 0) {
+      return;
+    }
+    FilterOperator filOp = (FilterOperator)curOp;
+    List<String> prunList = null;
+    List<Integer>[] childToParentIndex = null;
+
+    for (Operator<? extends OperatorDesc> child : curOp.getChildOperators()) {
+      if (child instanceof UnionOperator) {
+        prunList = prunedColLists.get(child);
+        if (prunList == null || prunList.size() == 0 || parentPrunList.size() == prunList.size()) {
+          continue;
+        }
+
+        ArrayList<ExprNodeDesc> exprs = new ArrayList<ExprNodeDesc>();
+        ArrayList<String> outputColNames = new ArrayList<String>();
+        Map<String, ExprNodeDesc> colExprMap = new HashMap<String, ExprNodeDesc>();
+        ArrayList<ColumnInfo> outputRS = new ArrayList<ColumnInfo>();
+        for (ColumnInfo colInfo : child.getSchema().getSignature()) {
+          if (!prunList.contains(colInfo.getInternalName())) {
+            continue;
+          }
+          ExprNodeDesc colDesc = new ExprNodeColumnDesc(colInfo.getType(),
+              colInfo.getInternalName(), colInfo.getTabAlias(), colInfo.getIsVirtualCol());
+          exprs.add(colDesc);
+          outputColNames.add(colInfo.getInternalName());
+          ColumnInfo newCol = new ColumnInfo(colInfo.getInternalName(), colInfo.getType(),
+                  colInfo.getTabAlias(), colInfo.getIsVirtualCol(), colInfo.isHiddenVirtualCol());
+          newCol.setAlias(colInfo.getAlias());
+          outputRS.add(newCol);
+          colExprMap.put(colInfo.getInternalName(), colDesc);
+        }
+        SelectDesc select = new SelectDesc(exprs, outputColNames, false);
+        curOp.removeChild(child);
+        SelectOperator sel = (SelectOperator) OperatorFactory.getAndMakeChild(
+            select, new RowSchema(outputRS), curOp);
+        OperatorFactory.makeChild(sel, child);
+        sel.setColumnExprMap(colExprMap);
+
+      }
+
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/c2dc452e/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java
index ac4236c..2dc15f9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java
@@ -108,7 +108,7 @@ public final class ColumnPrunerProcFactory {
           filterOpPrunedColListsOrderPreserved);
 
       pruneOperator(cppCtx, op, cppCtx.getPrunedColLists().get(op));
-
+      cppCtx.handleFilterUnionChildren(op);
       return null;
     }
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/c2dc452e/ql/src/test/queries/clientpositive/unionall_unbalancedppd.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/unionall_unbalancedppd.q b/ql/src/test/queries/clientpositive/unionall_unbalancedppd.q
new file mode 100644
index 0000000..0825c2d
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/unionall_unbalancedppd.q
@@ -0,0 +1,120 @@
+set hive.optimize.ppd=true;
+drop table if exists union_all_bug_test_1;
+drop table if exists union_all_bug_test_2;
+create table if not exists union_all_bug_test_1
+(
+f1 int,
+f2 int
+);
+
+create table if not exists union_all_bug_test_2
+(
+f1 int
+);
+
+explain SELECT f1
+FROM (
+
+SELECT
+f1
+, if('helloworld' like '%hello%' ,f1,f2) as filter
+FROM union_all_bug_test_1
+
+union all
+
+select
+f1
+, 0 as filter
+from union_all_bug_test_2
+) A
+WHERE (filter = 1);
+
+SELECT f1
+FROM (
+
+SELECT
+f1
+, if('helloworld' like '%hello%' ,f1,f2) as filter
+FROM union_all_bug_test_1
+
+union all
+
+select
+f1
+, 0 as filter
+from union_all_bug_test_2
+) A
+WHERE (filter = 1);
+
+insert into table union_all_bug_test_1 values (1,1);
+insert into table union_all_bug_test_2 values (1);
+insert into table union_all_bug_test_1 values (0,0);
+insert into table union_all_bug_test_2 values (0);
+
+SELECT f1
+FROM (
+
+SELECT
+f1
+, if('helloworld' like '%hello%' ,f1,f2) as filter
+FROM union_all_bug_test_1
+
+union all
+
+select
+f1
+, 0 as filter
+from union_all_bug_test_2
+) A
+WHERE (filter = 1);
+
+SELECT f1
+FROM (
+
+SELECT
+f1
+, if('helloworld' like '%hello%' ,f1,f2) as filter
+FROM union_all_bug_test_1
+
+union all
+
+select
+f1
+, 0 as filter
+from union_all_bug_test_2
+) A
+WHERE (filter = 0);
+
+SELECT f1
+FROM (
+
+SELECT
+f1
+, if('helloworld' like '%hello%' ,f1,f2) as filter
+FROM union_all_bug_test_1
+
+union all
+
+select
+f1
+, 0 as filter
+from union_all_bug_test_2
+) A
+WHERE (filter = 1 or filter = 0);
+
+SELECT f1
+FROM (
+
+SELECT
+f1
+, if('helloworld' like '%hello%' ,f1,f2) as filter
+FROM union_all_bug_test_1
+
+union all
+
+select
+f1
+, 0 as filter
+from union_all_bug_test_2
+) A
+WHERE (f1 = 1);

http://git-wip-us.apache.org/repos/asf/hive/blob/c2dc452e/ql/src/test/results/clientpositive/unionall_unbalancedppd.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/unionall_unbalancedppd.q.out b/ql/src/test/results/clientpositive/unionall_unbalancedppd.q.out
new file mode 100644
index 0000000..46828e9
--- /dev/null
+++ b/ql/src/test/results/clientpositive/unionall_unbalancedppd.q.out
@@ -0,0 +1,373 @@
+PREHOOK: query: drop table if exists union_all_bug_test_1
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists union_all_bug_test_1
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: drop table if exists union_all_bug_test_2
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists union_all_bug_test_2
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: create table if not exists union_all_bug_test_1
+(
+f1 int,
+f2 int
+)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@union_all_bug_test_1
+POSTHOOK: query: create table if not exists union_all_bug_test_1
+(
+f1 int,
+f2 int
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@union_all_bug_test_1
+PREHOOK: query: create table if not exists union_all_bug_test_2
+(
+f1 int
+)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@union_all_bug_test_2
+POSTHOOK: query: create table if not exists union_all_bug_test_2
+(
+f1 int
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@union_all_bug_test_2
+PREHOOK: query: explain SELECT f1
+FROM (
+
+SELECT
+f1
+, if('helloworld' like '%hello%' ,f1,f2) as filter
+FROM union_all_bug_test_1
+
+union all
+
+select
+f1
+, 0 as filter
+from union_all_bug_test_2
+) A
+WHERE (filter = 1)
+PREHOOK: type: QUERY
+POSTHOOK: query: explain SELECT f1
+FROM (
+
+SELECT
+f1
+, if('helloworld' like '%hello%' ,f1,f2) as filter
+FROM union_all_bug_test_1
+
+union all
+
+select
+f1
+, 0 as filter
+from union_all_bug_test_2
+) A
+WHERE (filter = 1)
+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: union_all_bug_test_1
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Filter Operator
+              predicate: (if(true, f1, f2) = 1) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: f1 (type: int)
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                Union
+                  Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  File Output Operator
+                    compressed: false
+                    Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL 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
+          TableScan
+            alias: union_all_bug_test_2
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+            Filter Operator
+              predicate: false (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+              Select Operator
+                expressions: f1 (type: int)
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                Union
+                  Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                  File Output Operator
+                    compressed: false
+                    Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                    table:
+                        input format: org.apache.hadoop.mapred.TextInputFormat
+                        output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: SELECT f1
+FROM (
+
+SELECT
+f1
+, if('helloworld' like '%hello%' ,f1,f2) as filter
+FROM union_all_bug_test_1
+
+union all
+
+select
+f1
+, 0 as filter
+from union_all_bug_test_2
+) A
+WHERE (filter = 1)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@union_all_bug_test_1
+PREHOOK: Input: default@union_all_bug_test_2
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT f1
+FROM (
+
+SELECT
+f1
+, if('helloworld' like '%hello%' ,f1,f2) as filter
+FROM union_all_bug_test_1
+
+union all
+
+select
+f1
+, 0 as filter
+from union_all_bug_test_2
+) A
+WHERE (filter = 1)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@union_all_bug_test_1
+POSTHOOK: Input: default@union_all_bug_test_2
+#### A masked pattern was here ####
+PREHOOK: query: insert into table union_all_bug_test_1 values (1,1)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__1
+PREHOOK: Output: default@union_all_bug_test_1
+POSTHOOK: query: insert into table union_all_bug_test_1 values (1,1)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__1
+POSTHOOK: Output: default@union_all_bug_test_1
+POSTHOOK: Lineage: union_all_bug_test_1.f1 EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+POSTHOOK: Lineage: union_all_bug_test_1.f2 EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+PREHOOK: query: insert into table union_all_bug_test_2 values (1)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__2
+PREHOOK: Output: default@union_all_bug_test_2
+POSTHOOK: query: insert into table union_all_bug_test_2 values (1)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__2
+POSTHOOK: Output: default@union_all_bug_test_2
+POSTHOOK: Lineage: union_all_bug_test_2.f1 EXPRESSION [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+PREHOOK: query: insert into table union_all_bug_test_1 values (0,0)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__3
+PREHOOK: Output: default@union_all_bug_test_1
+POSTHOOK: query: insert into table union_all_bug_test_1 values (0,0)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__3
+POSTHOOK: Output: default@union_all_bug_test_1
+POSTHOOK: Lineage: union_all_bug_test_1.f1 EXPRESSION [(values__tmp__table__3)values__tmp__table__3.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+POSTHOOK: Lineage: union_all_bug_test_1.f2 EXPRESSION [(values__tmp__table__3)values__tmp__table__3.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+PREHOOK: query: insert into table union_all_bug_test_2 values (0)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__4
+PREHOOK: Output: default@union_all_bug_test_2
+POSTHOOK: query: insert into table union_all_bug_test_2 values (0)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__4
+POSTHOOK: Output: default@union_all_bug_test_2
+POSTHOOK: Lineage: union_all_bug_test_2.f1 EXPRESSION [(values__tmp__table__4)values__tmp__table__4.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+PREHOOK: query: SELECT f1
+FROM (
+
+SELECT
+f1
+, if('helloworld' like '%hello%' ,f1,f2) as filter
+FROM union_all_bug_test_1
+
+union all
+
+select
+f1
+, 0 as filter
+from union_all_bug_test_2
+) A
+WHERE (filter = 1)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@union_all_bug_test_1
+PREHOOK: Input: default@union_all_bug_test_2
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT f1
+FROM (
+
+SELECT
+f1
+, if('helloworld' like '%hello%' ,f1,f2) as filter
+FROM union_all_bug_test_1
+
+union all
+
+select
+f1
+, 0 as filter
+from union_all_bug_test_2
+) A
+WHERE (filter = 1)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@union_all_bug_test_1
+POSTHOOK: Input: default@union_all_bug_test_2
+#### A masked pattern was here ####
+1
+PREHOOK: query: SELECT f1
+FROM (
+
+SELECT
+f1
+, if('helloworld' like '%hello%' ,f1,f2) as filter
+FROM union_all_bug_test_1
+
+union all
+
+select
+f1
+, 0 as filter
+from union_all_bug_test_2
+) A
+WHERE (filter = 0)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@union_all_bug_test_1
+PREHOOK: Input: default@union_all_bug_test_2
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT f1
+FROM (
+
+SELECT
+f1
+, if('helloworld' like '%hello%' ,f1,f2) as filter
+FROM union_all_bug_test_1
+
+union all
+
+select
+f1
+, 0 as filter
+from union_all_bug_test_2
+) A
+WHERE (filter = 0)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@union_all_bug_test_1
+POSTHOOK: Input: default@union_all_bug_test_2
+#### A masked pattern was here ####
+0
+1
+0
+PREHOOK: query: SELECT f1
+FROM (
+
+SELECT
+f1
+, if('helloworld' like '%hello%' ,f1,f2) as filter
+FROM union_all_bug_test_1
+
+union all
+
+select
+f1
+, 0 as filter
+from union_all_bug_test_2
+) A
+WHERE (filter = 1 or filter = 0)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@union_all_bug_test_1
+PREHOOK: Input: default@union_all_bug_test_2
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT f1
+FROM (
+
+SELECT
+f1
+, if('helloworld' like '%hello%' ,f1,f2) as filter
+FROM union_all_bug_test_1
+
+union all
+
+select
+f1
+, 0 as filter
+from union_all_bug_test_2
+) A
+WHERE (filter = 1 or filter = 0)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@union_all_bug_test_1
+POSTHOOK: Input: default@union_all_bug_test_2
+#### A masked pattern was here ####
+1
+0
+1
+0
+PREHOOK: query: SELECT f1
+FROM (
+
+SELECT
+f1
+, if('helloworld' like '%hello%' ,f1,f2) as filter
+FROM union_all_bug_test_1
+
+union all
+
+select
+f1
+, 0 as filter
+from union_all_bug_test_2
+) A
+WHERE (f1 = 1)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@union_all_bug_test_1
+PREHOOK: Input: default@union_all_bug_test_2
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT f1
+FROM (
+
+SELECT
+f1
+, if('helloworld' like '%hello%' ,f1,f2) as filter
+FROM union_all_bug_test_1
+
+union all
+
+select
+f1
+, 0 as filter
+from union_all_bug_test_2
+) A
+WHERE (f1 = 1)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@union_all_bug_test_1
+POSTHOOK: Input: default@union_all_bug_test_2
+#### A masked pattern was here ####
+1
+1


[16/43] hive git commit: HIVE-11077 Add support in parser and wire up to txn manager (Eugene Koifman, reviewed by Alan Gates)

Posted by xu...@apache.org.
HIVE-11077 Add support in parser and wire up to txn manager (Eugene Koifman, reviewed by Alan Gates)


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

Branch: refs/heads/spark
Commit: e57c3602b831340519d5d004cf4119da2f3e7ef8
Parents: 2240dbd
Author: Eugene Koifman <ek...@hortonworks.com>
Authored: Wed Jul 22 12:44:40 2015 -0700
Committer: Eugene Koifman <ek...@hortonworks.com>
Committed: Wed Jul 22 12:44:40 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hive/cli/TestOptionsProcessor.java   |   1 -
 .../hadoop/hive/common/ValidReadTxnList.java    |   2 +-
 .../hadoop/hive/metastore/txn/TxnHandler.java   |   9 +-
 .../metastore/txn/ValidCompactorTxnList.java    |   2 +-
 .../java/org/apache/hadoop/hive/ql/Context.java |   1 -
 .../java/org/apache/hadoop/hive/ql/Driver.java  | 196 +++++---
 .../org/apache/hadoop/hive/ql/ErrorMsg.java     |   2 +-
 .../org/apache/hadoop/hive/ql/QueryPlan.java    |  18 +-
 .../apache/hadoop/hive/ql/exec/MoveTask.java    |   2 +-
 .../hadoop/hive/ql/lockmgr/DbTxnManager.java    |  36 +-
 .../hadoop/hive/ql/lockmgr/DummyTxnManager.java |   8 +
 .../hadoop/hive/ql/lockmgr/HiveTxnManager.java  |  21 +
 .../hive/ql/lockmgr/HiveTxnManagerImpl.java     |  10 +
 .../hadoop/hive/ql/lockmgr/LockException.java   |   8 +-
 .../hadoop/hive/ql/metadata/HiveException.java  |   3 +
 .../hive/ql/parse/BaseSemanticAnalyzer.java     |  13 +
 .../org/apache/hadoop/hive/ql/parse/HiveLexer.g |  11 +
 .../apache/hadoop/hive/ql/parse/HiveParser.g    |  70 +++
 .../hadoop/hive/ql/parse/IdentifiersParser.g    |  19 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |  25 +-
 .../hive/ql/parse/SemanticAnalyzerFactory.java  |  12 +
 .../hadoop/hive/ql/plan/HiveOperation.java      |  32 +-
 .../ql/processors/CommandProcessorResponse.java |  21 +-
 .../hadoop/hive/ql/processors/HiveCommand.java  |   3 +
 .../authorization/plugin/HiveOperationType.java |   5 +
 .../plugin/sqlstd/Operation2Privilege.java      |  11 +
 .../hadoop/hive/ql/session/SessionState.java    |  34 +-
 .../apache/hadoop/hive/ql/TestTxnCommands.java  | 473 +++++++++++++++++++
 .../positive/TestTransactionStatement.java      | 102 ++++
 .../hive/ql/session/TestSessionState.java       |   2 +-
 .../clientnegative/exchange_partition.q.out     |   2 +-
 .../clientpositive/exchange_partition.q.out     |   4 +-
 .../clientpositive/exchange_partition2.q.out    |   4 +-
 .../clientpositive/exchange_partition3.q.out    |   4 +-
 34 files changed, 1020 insertions(+), 146 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/e57c3602/cli/src/test/org/apache/hadoop/hive/cli/TestOptionsProcessor.java
----------------------------------------------------------------------
diff --git a/cli/src/test/org/apache/hadoop/hive/cli/TestOptionsProcessor.java b/cli/src/test/org/apache/hadoop/hive/cli/TestOptionsProcessor.java
index 9d0399a..ac22ab1 100644
--- a/cli/src/test/org/apache/hadoop/hive/cli/TestOptionsProcessor.java
+++ b/cli/src/test/org/apache/hadoop/hive/cli/TestOptionsProcessor.java
@@ -56,7 +56,6 @@ public class TestOptionsProcessor {
     assertEquals("execString", sessionState.execString);
     assertEquals(0, sessionState.initFiles.size());
     assertTrue(sessionState.getIsVerbose());
-    sessionState.setConf(null);
     assertTrue(sessionState.getIsSilent());
 
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/e57c3602/common/src/java/org/apache/hadoop/hive/common/ValidReadTxnList.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/ValidReadTxnList.java b/common/src/java/org/apache/hadoop/hive/common/ValidReadTxnList.java
index 479e0df..fda242d 100644
--- a/common/src/java/org/apache/hadoop/hive/common/ValidReadTxnList.java
+++ b/common/src/java/org/apache/hadoop/hive/common/ValidReadTxnList.java
@@ -105,7 +105,7 @@ public class ValidReadTxnList implements ValidTxnList {
 
   @Override
   public void readFromString(String src) {
-    if (src == null) {
+    if (src == null || src.length() == 0) {
       highWatermark = Long.MAX_VALUE;
       exceptions = new long[0];
     } else {

http://git-wip-us.apache.org/repos/asf/hive/blob/e57c3602/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
index fd9c275..c0e83c6 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
@@ -380,8 +380,9 @@ public class TxnHandler {
           "tc_partition from TXN_COMPONENTS where tc_txnid = " + txnid;
         LOG.debug("Going to execute insert <" + s + ">");
         if (stmt.executeUpdate(s) < 1) {
-          LOG.warn("Expected to move at least one record from txn_components to " +
-            "completed_txn_components when committing txn!");
+          //this can be reasonable for an empty txn START/COMMIT
+          LOG.info("Expected to move at least one record from txn_components to " +
+            "completed_txn_components when committing txn! txnid:" + txnid);
         }
 
         // Always access TXN_COMPONENTS before HIVE_LOCKS;
@@ -1351,7 +1352,7 @@ public class TxnHandler {
     throws NoSuchTxnException,  TxnAbortedException, MetaException, SQLException {
     // We want to minimize the number of concurrent lock requests being issued.  If we do not we
     // get a large number of deadlocks in the database, since this method has to both clean
-    // timedout locks and insert new locks.  This synchronization barrier will not eliminiate all
+    // timedout locks and insert new locks.  This synchronization barrier will not eliminate all
     // deadlocks, and the code is still resilient in the face of a database deadlock.  But it
     // will reduce the number.  This could have been done via a lock table command in the
     // underlying database, but was not for two reasons.  One, different databases have different
@@ -1452,7 +1453,7 @@ public class TxnHandler {
                                  long extLockId,
                                  boolean alwaysCommit)
     throws NoSuchLockException, NoSuchTxnException, TxnAbortedException, MetaException, SQLException {
-    List<LockInfo> locksBeingChecked = getLockInfoFromLockId(dbConn, extLockId);
+    List<LockInfo> locksBeingChecked = getLockInfoFromLockId(dbConn, extLockId);//being acquired now
     LockResponse response = new LockResponse();
     response.setLockid(extLockId);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/e57c3602/metastore/src/java/org/apache/hadoop/hive/metastore/txn/ValidCompactorTxnList.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/ValidCompactorTxnList.java b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/ValidCompactorTxnList.java
index 71f14e5..67631ba 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/ValidCompactorTxnList.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/ValidCompactorTxnList.java
@@ -88,7 +88,7 @@ public class ValidCompactorTxnList extends ValidReadTxnList {
 
   @Override
   public void readFromString(String src) {
-    if (src == null) {
+    if (src == null || src.length() == 0) {
       highWatermark = Long.MAX_VALUE;
       exceptions = new long[0];
     } else {

http://git-wip-us.apache.org/repos/asf/hive/blob/e57c3602/ql/src/java/org/apache/hadoop/hive/ql/Context.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Context.java b/ql/src/java/org/apache/hadoop/hive/ql/Context.java
index a74bbbe..ca0d487 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Context.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Context.java
@@ -96,7 +96,6 @@ public class Context {
 
   // List of Locks for this query
   protected List<HiveLock> hiveLocks;
-  protected HiveLockManager hiveLockMgr;
 
   // Transaction manager for this query
   protected HiveTxnManager hiveTxnManager;

http://git-wip-us.apache.org/repos/asf/hive/blob/e57c3602/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
index 6ad3f49..424f4fa 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
@@ -387,7 +387,10 @@ public class Driver implements CommandProcessor {
     SessionState.get().setupQueryCurrentTimestamp();
 
     try {
-      command = new VariableSubstitution().substitute(conf,command);
+      // Initialize the transaction manager.  This must be done before analyze is called.
+      SessionState.get().initTxnMgr(conf);
+
+      command = new VariableSubstitution().substitute(conf, command);
       ctx = new Context(conf);
       ctx.setTryCount(getTryCount());
       ctx.setCmd(command);
@@ -399,13 +402,6 @@ public class Driver implements CommandProcessor {
       tree = ParseUtils.findRootNonNullToken(tree);
       perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.PARSE);
 
-      // Initialize the transaction manager.  This must be done before analyze is called.  Also
-      // record the valid transactions for this query.  We have to do this at compile time
-      // because we use the information in planning the query.  Also,
-      // we want to record it at this point so that users see data valid at the point that they
-      // submit the query.
-      SessionState.get().initTxnMgr(conf);
-      recordValidTxns();
 
       perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.ANALYZE);
       BaseSemanticAnalyzer sem = SemanticAnalyzerFactory.get(conf, tree);
@@ -445,10 +441,8 @@ public class Driver implements CommandProcessor {
       // to avoid returning sensitive data
       String queryStr = HookUtils.redactLogString(conf, command);
 
-      String operationName = ctx.getExplain() ?
-        HiveOperation.EXPLAIN.getOperationName() : SessionState.get().getCommandType();
       plan = new QueryPlan(queryStr, sem, perfLogger.getStartTime(PerfLogger.DRIVER_RUN), queryId,
-        operationName, getSchema(sem, conf));
+        SessionState.get().getHiveOperation(), getSchema(sem, conf));
 
       conf.setVar(HiveConf.ConfVars.HIVEQUERYSTRING, queryStr);
 
@@ -507,7 +501,8 @@ public class Driver implements CommandProcessor {
       downstreamError = e;
       console.printError(errorMessage, "\n"
           + org.apache.hadoop.util.StringUtils.stringifyException(e));
-      return error.getErrorCode();
+      return error.getErrorCode();//todo: this is bad if returned as cmd shell exit
+      // since it exceeds valid range of shell return values
     } finally {
       perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.COMPILE);
       dumpMetaCallTimingWithoutEx("compilation");
@@ -937,30 +932,32 @@ public class Driver implements CommandProcessor {
   // Write the current set of valid transactions into the conf file so that it can be read by
   // the input format.
   private void recordValidTxns() throws LockException {
-    ValidTxnList txns = SessionState.get().getTxnMgr().getValidTxns();
+    HiveTxnManager txnMgr = SessionState.get().getTxnMgr();
+    ValidTxnList txns = txnMgr.getValidTxns();
     String txnStr = txns.toString();
     conf.set(ValidTxnList.VALID_TXNS_KEY, txnStr);
-    LOG.debug("Encoding valid txns info " + txnStr);
-    // TODO I think when we switch to cross query transactions we need to keep this list in
-    // session state rather than agressively encoding it in the conf like this.  We can let the
-    // TableScanOperators then encode it in the conf before calling the input formats.
+    LOG.debug("Encoding valid txns info " + txnStr + " txnid:" + txnMgr.getCurrentTxnId());
   }
 
   /**
    * Acquire read and write locks needed by the statement. The list of objects to be locked are
-   * obtained from the inputs and outputs populated by the compiler. The lock acuisition scheme is
+   * obtained from the inputs and outputs populated by the compiler. The lock acquisition scheme is
    * pretty simple. If all the locks cannot be obtained, error out. Deadlock is avoided by making
    * sure that the locks are lexicographically sorted.
    *
    * This method also records the list of valid transactions.  This must be done after any
    * transactions have been opened and locks acquired.
+   * @param startTxnImplicitly in AC=false, the 1st DML starts a txn
    **/
-  private int acquireLocksAndOpenTxn() {
+  private int acquireLocksAndOpenTxn(boolean startTxnImplicitly) {
     PerfLogger perfLogger = PerfLogger.getPerfLogger();
     perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.ACQUIRE_READ_WRITE_LOCKS);
 
     SessionState ss = SessionState.get();
     HiveTxnManager txnMgr = ss.getTxnMgr();
+    if(startTxnImplicitly) {
+      assert !txnMgr.getAutoCommit();
+    }
 
     try {
       // Don't use the userName member, as it may or may not have been set.  Get the value from
@@ -976,27 +973,34 @@ public class Driver implements CommandProcessor {
             "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e));
         return 10;
       }
-      if (acidSinks != null && acidSinks.size() > 0) {
+
+      boolean existingTxn = txnMgr.isTxnOpen();
+      if((txnMgr.getAutoCommit() && haveAcidWrite()) || plan.getOperation() == HiveOperation.START_TRANSACTION ||
+        (!txnMgr.getAutoCommit() && startTxnImplicitly)) {
         // We are writing to tables in an ACID compliant way, so we need to open a transaction
-        long txnId = ss.getCurrentTxn();
-        if (txnId == SessionState.NO_CURRENT_TXN) {
-          txnId = txnMgr.openTxn(userFromUGI);
-          ss.setCurrentTxn(txnId);
-          LOG.debug("Setting current transaction to " + txnId);
-        }
-        // Set the transaction id in all of the acid file sinks
-        if (acidSinks != null) {
-          for (FileSinkDesc desc : acidSinks) {
-            desc.setTransactionId(txnId);
-            desc.setStatementId(txnMgr.getStatementId());
-          }
+        txnMgr.openTxn(userFromUGI);
+      }
+      // Set the transaction id in all of the acid file sinks
+      if (haveAcidWrite()) {
+        for (FileSinkDesc desc : acidSinks) {
+          desc.setTransactionId(txnMgr.getCurrentTxnId());
+          desc.setStatementId(txnMgr.getStatementId());
         }
-
-        // TODO Once we move to cross query transactions we need to add the open transaction to
-        // our list of valid transactions.  We don't have a way to do that right now.
       }
-
+      /*Note, we have to record snapshot after lock acquisition to prevent lost update problem
+      consider 2 concurrent "update table T set x = x + 1".  1st will get the locks and the
+      2nd will block until 1st one commits and only then lock in the snapshot, i.e. it will
+      see the changes made by 1st one.  This takes care of autoCommit=true case.
+      For multi-stmt txns this is not sufficient and will be managed via WriteSet tracking
+      in the lock manager.*/
       txnMgr.acquireLocks(plan, ctx, userFromUGI);
+      if(!existingTxn) {
+        //For multi-stmt txns we should record the snapshot when txn starts but
+        // don't update it after that until txn completes.  Thus the check for {@code existingTxn}
+        //For autoCommit=true, Read-only statements, txn is implicit, i.e. lock in the snapshot
+        //for each statement.
+        recordValidTxns();
+      }
 
       return 0;
     } catch (LockException e) {
@@ -1011,6 +1015,9 @@ public class Driver implements CommandProcessor {
     }
   }
 
+  private boolean haveAcidWrite() {
+    return acidSinks != null && !acidSinks.isEmpty();
+  }
   /**
    * @param hiveLocks
    *          list of hive locks to be released Release all the locks specified. If some of the
@@ -1028,17 +1035,14 @@ public class Driver implements CommandProcessor {
     HiveTxnManager txnMgr = ss.getTxnMgr();
     // If we've opened a transaction we need to commit or rollback rather than explicitly
     // releasing the locks.
-    if (ss.getCurrentTxn() != SessionState.NO_CURRENT_TXN && ss.isAutoCommit()) {
-      try {
-        if (commit) {
-          txnMgr.commitTxn();
-        } else {
-          txnMgr.rollbackTxn();
-        }
-      } finally {
-        ss.setCurrentTxn(SessionState.NO_CURRENT_TXN);
+    if (txnMgr.isTxnOpen()) {
+      if (commit) {
+        txnMgr.commitTxn();//both commit & rollback clear ALL locks for this tx
+      } else {
+        txnMgr.rollbackTxn();
       }
     } else {
+      //since there is no tx, we only have locks for current query (if any)
       if (hiveLocks != null) {
         txnMgr.getLockManager().releaseLocks(hiveLocks);
       }
@@ -1193,44 +1197,77 @@ public class Driver implements CommandProcessor {
       // Since we're reusing the compiled plan, we need to update its start time for current run
       plan.setQueryStartTime(perfLogger.getStartTime(PerfLogger.DRIVER_RUN));
     }
-
     // the reason that we set the txn manager for the cxt here is because each
     // query has its own ctx object. The txn mgr is shared across the
     // same instance of Driver, which can run multiple queries.
-    ctx.setHiveTxnManager(SessionState.get().getTxnMgr());
+    HiveTxnManager txnManager = SessionState.get().getTxnMgr();
+    ctx.setHiveTxnManager(txnManager);
+
+    boolean startTxnImplicitly = false;
+    {
+      //this block ensures op makes sense in given context, e.g. COMMIT is valid only if txn is open
+      //DDL is not allowed in a txn, etc.
+      //an error in an open txn does a rollback of the txn
+      if (txnManager.isTxnOpen() && !plan.getOperation().isAllowedInTransaction()) {
+        assert !txnManager.getAutoCommit() : "didn't expect AC=true";
+        return rollback(new CommandProcessorResponse(12, ErrorMsg.OP_NOT_ALLOWED_IN_TXN, null,
+          plan.getOperationName(), Long.toString(txnManager.getCurrentTxnId())));
+      }
+      if(!txnManager.isTxnOpen() && plan.getOperation().isRequiresOpenTransaction()) {
+        return rollback(new CommandProcessorResponse(12, ErrorMsg.OP_NOT_ALLOWED_WITHOUT_TXN, null, plan.getOperationName()));
+      }
+      if(!txnManager.isTxnOpen() && plan.getOperation() == HiveOperation.QUERY && !txnManager.getAutoCommit()) {
+        //this effectively makes START TRANSACTION optional and supports JDBC setAutoCommit(false) semantics
+        //also, indirectly allows DDL to be executed outside a txn context 
+        startTxnImplicitly = true;
+      }
+      if(txnManager.getAutoCommit() && plan.getOperation() == HiveOperation.START_TRANSACTION) {
+          return rollback(new CommandProcessorResponse(12, ErrorMsg.OP_NOT_ALLOWED_IN_AUTOCOMMIT, null, plan.getOperationName()));
+      }
+    }
+    if(plan.getOperation() == HiveOperation.SET_AUTOCOMMIT) {
+      try {
+        if(plan.getAutoCommitValue() && !txnManager.getAutoCommit()) {
+          /*here, if there is an open txn, we want to commit it; this behavior matches
+          * https://docs.oracle.com/javase/6/docs/api/java/sql/Connection.html#setAutoCommit(boolean)*/
+          releaseLocksAndCommitOrRollback(ctx.getHiveLocks(), true);
+          txnManager.setAutoCommit(true);
+        }
+        else if(!plan.getAutoCommitValue() && txnManager.getAutoCommit()) {
+          txnManager.setAutoCommit(false);
+        }
+        else {/*didn't change autoCommit value - no-op*/}
+      }
+      catch(LockException e) {
+        return handleHiveException(e, 12);
+      }
+    }
 
     if (requiresLock()) {
-      ret = acquireLocksAndOpenTxn();
+      ret = acquireLocksAndOpenTxn(startTxnImplicitly);
       if (ret != 0) {
-        try {
-          releaseLocksAndCommitOrRollback(ctx.getHiveLocks(), false);
-        } catch (LockException e) {
-          // Not much to do here
-        }
-        return createProcessorResponse(ret);
+        return rollback(createProcessorResponse(ret));
       }
     }
     ret = execute();
     if (ret != 0) {
       //if needRequireLock is false, the release here will do nothing because there is no lock
-      try {
-        releaseLocksAndCommitOrRollback(ctx.getHiveLocks(), false);
-      } catch (LockException e) {
-        // Nothing to do here
-      }
-      return createProcessorResponse(ret);
+      return rollback(createProcessorResponse(ret));
     }
 
     //if needRequireLock is false, the release here will do nothing because there is no lock
     try {
-      releaseLocksAndCommitOrRollback(ctx.getHiveLocks(), true);
+      if(txnManager.getAutoCommit() || plan.getOperation() == HiveOperation.COMMIT) {
+        releaseLocksAndCommitOrRollback(ctx.getHiveLocks(), true);
+      }
+      else if(plan.getOperation() == HiveOperation.ROLLBACK) {
+        releaseLocksAndCommitOrRollback(ctx.getHiveLocks(), false);
+      }
+      else {
+        //txn (if there is one started) is not finished
+      }
     } catch (LockException e) {
-      errorMessage = "FAILED: Hive Internal Error: " + Utilities.getNameMessage(e);
-      SQLState = ErrorMsg.findSQLState(e.getMessage());
-      downstreamError = e;
-      console.printError(errorMessage + "\n"
-          + org.apache.hadoop.util.StringUtils.stringifyException(e));
-      return createProcessorResponse(12);
+      return handleHiveException(e, 12);
     }
 
     perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.DRIVER_RUN);
@@ -1253,6 +1290,31 @@ public class Driver implements CommandProcessor {
     return createProcessorResponse(ret);
   }
 
+  private CommandProcessorResponse rollback(CommandProcessorResponse cpr) {
+    try {
+      releaseLocksAndCommitOrRollback(ctx.getHiveLocks(), false);
+    }
+    catch (LockException e) {
+      LOG.error("rollback() FAILED: " + cpr);//make sure not to loose 
+      handleHiveException(e, 12, "Additional info in hive.log at \"rollback() FAILED\"");
+    }
+    return cpr;
+  }
+  private CommandProcessorResponse handleHiveException(HiveException e, int ret) {
+    return handleHiveException(e, ret, null);
+  }
+  private CommandProcessorResponse handleHiveException(HiveException e, int ret, String rootMsg) {
+    errorMessage = "FAILED: Hive Internal Error: " + Utilities.getNameMessage(e);
+    if(rootMsg != null) {
+      errorMessage += "\n" + rootMsg;
+    }
+    SQLState = e.getCanonicalErrorMsg() != null ?
+      e.getCanonicalErrorMsg().getSQLState() : ErrorMsg.findSQLState(e.getMessage());
+    downstreamError = e;
+    console.printError(errorMessage + "\n"
+      + org.apache.hadoop.util.StringUtils.stringifyException(e));
+    return createProcessorResponse(ret);
+  }
   private boolean requiresLock() {
     if (!checkConcurrency()) {
       return false;

http://git-wip-us.apache.org/repos/asf/hive/blob/e57c3602/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java b/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
index fbdd66a..39b287a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
@@ -447,7 +447,7 @@ public enum ErrorMsg {
       " (={2}). This is controlled by hive.limit.query.max.table.partition.", true),
   OP_NOT_ALLOWED_IN_AUTOCOMMIT(20006, "Operation {0} is not allowed when autoCommit=true.", true),//todo: better SQLState?
   OP_NOT_ALLOWED_IN_TXN(20007, "Operation {0} is not allowed in a transaction.  TransactionID={1}.", true),
-  OP_NOT_ALLOWED_WITHOUT_TXN(2008, "Operation {0} is not allowed since autoCommit=false and there is no active transaction", true),
+  OP_NOT_ALLOWED_WITHOUT_TXN(20008, "Operation {0} is not allowed since autoCommit=false and there is no active transaction", true),
 
   //========================== 30000 range starts here ========================//
   STATSPUBLISHER_NOT_OBTAINED(30000, "StatsPublisher cannot be obtained. " +

http://git-wip-us.apache.org/repos/asf/hive/blob/e57c3602/ql/src/java/org/apache/hadoop/hive/ql/QueryPlan.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/QueryPlan.java b/ql/src/java/org/apache/hadoop/hive/ql/QueryPlan.java
index 29a3939..b9776ea 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/QueryPlan.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/QueryPlan.java
@@ -50,6 +50,7 @@ import org.apache.hadoop.hive.ql.hooks.WriteEntity;
 import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer;
 import org.apache.hadoop.hive.ql.parse.ColumnAccessInfo;
 import org.apache.hadoop.hive.ql.parse.TableAccessInfo;
+import org.apache.hadoop.hive.ql.plan.HiveOperation;
 import org.apache.hadoop.hive.ql.plan.OperatorDesc;
 import org.apache.hadoop.hive.ql.plan.ReducerTimeStatsPerJob;
 import org.apache.hadoop.hive.ql.plan.api.AdjacencyType;
@@ -106,14 +107,16 @@ public class QueryPlan implements Serializable {
   private QueryProperties queryProperties;
 
   private transient Long queryStartTime;
-  private String operationName;
+  private final HiveOperation operation;
+  private Boolean autoCommitValue;
 
   public QueryPlan() {
     this.reducerTimeStatsPerJobList = new ArrayList<ReducerTimeStatsPerJob>();
+    operation = null;
   }
 
   public QueryPlan(String queryString, BaseSemanticAnalyzer sem, Long startTime, String queryId,
-      String operationName, Schema resultSchema) {
+                   HiveOperation operation, Schema resultSchema) {
     this.queryString = queryString;
 
     rootTasks = new ArrayList<Task<? extends Serializable>>();
@@ -134,7 +137,8 @@ public class QueryPlan implements Serializable {
     query.putToQueryAttributes("queryString", this.queryString);
     queryProperties = sem.getQueryProperties();
     queryStartTime = startTime;
-    this.operationName = operationName;
+    this.operation = operation;
+    this.autoCommitValue = sem.getAutoCommitValue();
     this.resultSchema = resultSchema;
   }
 
@@ -794,6 +798,12 @@ public class QueryPlan implements Serializable {
   }
 
   public String getOperationName() {
-    return operationName;
+    return operation == null ? null : operation.getOperationName();
+  }
+  public HiveOperation getOperation() {
+    return operation;
+  }
+  public Boolean getAutoCommitValue() {
+    return autoCommitValue;
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/e57c3602/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
index b07a37a..0a466e4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
@@ -380,7 +380,7 @@ public class MoveTask extends Task<MoveWork> implements Serializable {
                 tbd.getHoldDDLTime(),
                 isSkewedStoredAsDirs(tbd),
                 work.getLoadTableWork().getWriteType() != AcidUtils.Operation.NOT_ACID,
-                SessionState.get().getCurrentTxn());
+                SessionState.get().getTxnMgr().getCurrentTxnId());
             console.printInfo("\t Time taken for load dynamic partitions : "  +
                 (System.currentTimeMillis() - startTime));
 

http://git-wip-us.apache.org/repos/asf/hive/blob/e57c3602/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
index 445f606..4813d5b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
@@ -21,7 +21,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.common.ValidTxnList;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.metastore.LockComponentBuilder;
 import org.apache.hadoop.hive.metastore.LockRequestBuilder;
@@ -51,6 +50,10 @@ public class DbTxnManager extends HiveTxnManagerImpl {
 
   private DbLockManager lockMgr = null;
   private IMetaStoreClient client = null;
+  /**
+   * The Metastore NEXT_TXN_ID.NTXN_NEXT is initialized to 1; it contains the next available
+   * transaction id.  Thus is 1 is first transaction id.
+   */
   private long txnId = 0;
   /**
    * assigns a unique monotonically increasing ID to each statement
@@ -75,14 +78,16 @@ public class DbTxnManager extends HiveTxnManagerImpl {
   @Override
   public long openTxn(String user) throws LockException {
     init();
+    if(isTxnOpen()) {
+      throw new LockException("Transaction already opened. txnId=" + txnId);//ToDo: ErrorMsg
+    }
     try {
       txnId = client.openTxn(user);
       statementId = 0;
       LOG.debug("Opened txn " + txnId);
       return txnId;
     } catch (TException e) {
-      throw new LockException(ErrorMsg.METASTORE_COMMUNICATION_FAILED.getMsg(),
-          e);
+      throw new LockException(e, ErrorMsg.METASTORE_COMMUNICATION_FAILED);
     }
   }
 
@@ -232,7 +237,7 @@ public class DbTxnManager extends HiveTxnManagerImpl {
     }
 
     List<HiveLock> locks = new ArrayList<HiveLock>(1);
-    if(txnId > 0) {
+    if(isTxnOpen()) {
       statementId++;
     }
     LockState lockState = lockMgr.lock(rqstBuilder.build(), plan.getQueryId(), isBlocking, locks);
@@ -242,9 +247,8 @@ public class DbTxnManager extends HiveTxnManagerImpl {
 
   @Override
   public void commitTxn() throws LockException {
-    if (txnId == 0) {
-      throw new RuntimeException("Attempt to commit before opening a " +
-          "transaction");
+    if (!isTxnOpen()) {
+      throw new RuntimeException("Attempt to commit before opening a transaction");
     }
     try {
       lockMgr.clearLocalLockRecords();
@@ -267,9 +271,8 @@ public class DbTxnManager extends HiveTxnManagerImpl {
 
   @Override
   public void rollbackTxn() throws LockException {
-    if (txnId == 0) {
-      throw new RuntimeException("Attempt to rollback before opening a " +
-          "transaction");
+    if (!isTxnOpen()) {
+      throw new RuntimeException("Attempt to rollback before opening a transaction");
     }
     try {
       lockMgr.clearLocalLockRecords();
@@ -292,7 +295,7 @@ public class DbTxnManager extends HiveTxnManagerImpl {
     LOG.debug("Heartbeating lock and transaction " + txnId);
     List<HiveLock> locks = lockMgr.getLocks(false, false);
     if (locks.size() == 0) {
-      if (txnId == 0) {
+      if (!isTxnOpen()) {
         // No locks, no txn, we outta here.
         return;
       } else {
@@ -350,7 +353,7 @@ public class DbTxnManager extends HiveTxnManagerImpl {
   @Override
   protected void destruct() {
     try {
-      if (txnId > 0) rollbackTxn();
+      if (isTxnOpen()) rollbackTxn();
       if (lockMgr != null) lockMgr.close();
     } catch (Exception e) {
       LOG.error("Caught exception " + e.getClass().getName() + " with message <" + e.getMessage()
@@ -376,8 +379,15 @@ public class DbTxnManager extends HiveTxnManagerImpl {
     }
   }
   @Override
+  public boolean isTxnOpen() {
+    return txnId > 0;
+  }
+  @Override
+  public long getCurrentTxnId() {
+    return txnId;
+  }
+  @Override
   public int getStatementId() {
     return statementId;
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/e57c3602/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java
index 1906982..be5a593 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java
@@ -52,6 +52,14 @@ class DummyTxnManager extends HiveTxnManagerImpl {
     // No-op
     return 0L;
   }
+  @Override
+  public boolean isTxnOpen() {
+    return false;
+  }
+  @Override
+  public long getCurrentTxnId() {
+    return 0L;
+  }
 
   @Override
   public int getStatementId() {

http://git-wip-us.apache.org/repos/asf/hive/blob/e57c3602/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveTxnManager.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveTxnManager.java b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveTxnManager.java
index c900548..74512d7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveTxnManager.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveTxnManager.java
@@ -173,6 +173,27 @@ public interface HiveTxnManager {
    */
   boolean supportsAcid();
 
+  /**
+   * This behaves exactly as
+   * https://docs.oracle.com/javase/6/docs/api/java/sql/Connection.html#setAutoCommit(boolean)
+   */
+  void setAutoCommit(boolean autoCommit) throws LockException;
+
+  /**
+   * This behaves exactly as
+   * https://docs.oracle.com/javase/6/docs/api/java/sql/Connection.html#getAutoCommit()
+   */
+  boolean getAutoCommit();
+
+  boolean isTxnOpen();
+  /**
+   * if {@code isTxnOpen()}, returns the currently active transaction ID
+   */
+  long getCurrentTxnId();
+
+  /**
+   * 0..N Id of current statement within currently opened transaction
+   */
   int getStatementId();
 
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/e57c3602/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveTxnManagerImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveTxnManagerImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveTxnManagerImpl.java
index ceeae68..ed022d9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveTxnManagerImpl.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveTxnManagerImpl.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.hive.ql.plan.UnlockTableDesc;
 abstract class HiveTxnManagerImpl implements HiveTxnManager {
 
   protected HiveConf conf;
+  private boolean isAutoCommit = true;//true by default; matches JDBC spec
 
   void setHiveConf(HiveConf c) {
     conf = c;
@@ -58,6 +59,15 @@ abstract class HiveTxnManagerImpl implements HiveTxnManager {
   protected void finalize() throws Throwable {
     destruct();
   }
+  @Override
+  public void setAutoCommit(boolean autoCommit) throws LockException {
+    isAutoCommit = autoCommit;
+  }
+
+  @Override
+  public boolean getAutoCommit() {
+    return isAutoCommit;
+  }
 
   @Override
   public int lockTable(Hive db, LockTableDesc lockTbl) throws HiveException {

http://git-wip-us.apache.org/repos/asf/hive/blob/e57c3602/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/LockException.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/LockException.java b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/LockException.java
index 9894a70..8ea457e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/LockException.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/LockException.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hive.ql.lockmgr;
 
+import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
@@ -43,5 +44,10 @@ public class LockException extends HiveException {
   public LockException(String message, Throwable cause) {
     super(message, cause);
   }
-
+  public LockException(Throwable cause, ErrorMsg errorMsg, String... msgArgs) {
+    super(cause, errorMsg, msgArgs);
+  }
+  public LockException(Throwable cause, ErrorMsg errorMsg) {
+    super(cause, errorMsg);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/e57c3602/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveException.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveException.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveException.java
index 1d895ca..d017705 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveException.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveException.java
@@ -60,6 +60,9 @@ public class HiveException extends Exception {
     canonicalErrorMsg = errorMsg;
 
   }
+  public HiveException(Throwable cause, ErrorMsg errorMsg) {
+    this(cause, errorMsg, new String[0]);
+  }
   /**
    * @return {@link ErrorMsg#GENERIC_ERROR} by default
    */

http://git-wip-us.apache.org/repos/asf/hive/blob/e57c3602/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
index d72991f..fbe93f9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
@@ -125,6 +125,19 @@ public abstract class BaseSemanticAnalyzer {
    * Columns accessed by updates
    */
   protected ColumnAccessInfo updateColumnAccessInfo;
+  /**
+   * the value of set autocommit true|false
+   * It's an object to make sure it's {@code null} if the parsed statement is
+   * not 'set autocommit...'
+   */
+  private Boolean autoCommitValue;
+
+  public Boolean getAutoCommitValue() {
+    return autoCommitValue;
+  }
+  void setAutoCommitValue(Boolean autoCommit) {
+    autoCommitValue = autoCommit;
+  }
 
 
   public boolean skipAuthorization() {

http://git-wip-us.apache.org/repos/asf/hive/blob/e57c3602/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
index 85c0ae6..9f8cfd1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
@@ -301,6 +301,17 @@ KW_DAY: 'DAY';
 KW_HOUR: 'HOUR';
 KW_MINUTE: 'MINUTE';
 KW_SECOND: 'SECOND';
+KW_START: 'START';
+KW_TRANSACTION: 'TRANSACTION';
+KW_COMMIT: 'COMMIT';
+KW_ROLLBACK: 'ROLLBACK';
+KW_WORK: 'WORK';
+KW_ONLY: 'ONLY';
+KW_WRITE: 'WRITE';
+KW_ISOLATION: 'ISOLATION';
+KW_LEVEL: 'LEVEL';
+KW_SNAPSHOT: 'SNAPSHOT';
+KW_AUTOCOMMIT: 'AUTOCOMMIT';
 
 // Operators
 // NOTE: if you add a new function/operator, add it to sysFuncNames so that describe function _FUNC_ will work.

http://git-wip-us.apache.org/repos/asf/hive/blob/e57c3602/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
index 3f95bb8..cf7ab3a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
@@ -354,6 +354,15 @@ TOK_ANONYMOUS;
 TOK_COL_NAME;
 TOK_URI_TYPE;
 TOK_SERVER_TYPE;
+TOK_START_TRANSACTION;
+TOK_ISOLATION_LEVEL;
+TOK_ISOLATION_SNAPSHOT;
+TOK_TXN_ACCESS_MODE;
+TOK_TXN_READ_ONLY;
+TOK_TXN_READ_WRITE;
+TOK_COMMIT;
+TOK_ROLLBACK;
+TOK_SET_AUTOCOMMIT;
 }
 
 
@@ -375,6 +384,7 @@ import org.apache.hadoop.hive.conf.HiveConf;
 
   private static HashMap<String, String> xlateMap;
   static {
+    //this is used to support auto completion in CLI
     xlateMap = new HashMap<String, String>();
 
     // Keywords
@@ -693,6 +703,7 @@ execStatement
     | ddlStatement
     | deleteStatement
     | updateStatement
+    | sqlTransactionStatement
     ;
 
 loadStatement
@@ -2363,3 +2374,62 @@ updateStatement
    :
    KW_UPDATE tableName setColumnsClause whereClause? -> ^(TOK_UPDATE_TABLE tableName setColumnsClause whereClause?)
    ;
+
+/*
+BEGIN user defined transaction boundaries; follows SQL 2003 standard exactly except for addition of
+"setAutoCommitStatement" which is not in the standard doc but is supported by most SQL engines.
+*/
+sqlTransactionStatement
+@init { pushMsg("transaction statement", state); }
+@after { popMsg(state); }
+  :
+  startTransactionStatement
+	|	commitStatement
+	|	rollbackStatement
+	| setAutoCommitStatement
+	;
+
+startTransactionStatement
+  :
+  KW_START KW_TRANSACTION ( transactionMode  ( COMMA transactionMode  )* )? -> ^(TOK_START_TRANSACTION transactionMode*)
+  ;
+
+transactionMode
+  :
+  isolationLevel
+  | transactionAccessMode -> ^(TOK_TXN_ACCESS_MODE transactionAccessMode)
+  ;
+
+transactionAccessMode
+  :
+  KW_READ KW_ONLY -> TOK_TXN_READ_ONLY
+  | KW_READ KW_WRITE -> TOK_TXN_READ_WRITE
+  ;
+
+isolationLevel
+  :
+  KW_ISOLATION KW_LEVEL levelOfIsolation -> ^(TOK_ISOLATION_LEVEL levelOfIsolation)
+  ;
+
+/*READ UNCOMMITTED | READ COMMITTED | REPEATABLE READ | SERIALIZABLE may be supported later*/
+levelOfIsolation
+  :
+  KW_SNAPSHOT -> TOK_ISOLATION_SNAPSHOT
+  ;
+
+commitStatement
+  :
+  KW_COMMIT ( KW_WORK )? -> TOK_COMMIT
+  ;
+
+rollbackStatement
+  :
+  KW_ROLLBACK ( KW_WORK )? -> TOK_ROLLBACK
+  ;
+setAutoCommitStatement
+  :
+  KW_SET KW_AUTOCOMMIT booleanValueTok -> ^(TOK_SET_AUTOCOMMIT booleanValueTok)
+  ;
+/*
+END user defined transaction boundaries
+*/

http://git-wip-us.apache.org/repos/asf/hive/blob/e57c3602/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
index 4f8be52..501287d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
@@ -499,6 +499,12 @@ booleanValue
     KW_TRUE^ | KW_FALSE^
     ;
 
+booleanValueTok
+   :
+   KW_TRUE -> TOK_TRUE
+   | KW_FALSE -> TOK_FALSE
+   ;
+
 tableOrPartition
    :
    tableName partitionSpec? -> ^(TOK_TAB tableName partitionSpec?)
@@ -629,7 +635,18 @@ nonReserved
     | KW_STREAMTABLE | KW_STRING | KW_STRUCT | KW_TABLES | KW_TBLPROPERTIES | KW_TEMPORARY | KW_TERMINATED
     | KW_TINYINT | KW_TOUCH | KW_TRANSACTIONS | KW_UNARCHIVE | KW_UNDO | KW_UNIONTYPE | KW_UNLOCK | KW_UNSET
     | KW_UNSIGNED | KW_URI | KW_USE | KW_UTC | KW_UTCTIMESTAMP | KW_VALUE_TYPE | KW_VIEW | KW_WHILE | KW_YEAR
-    ;
+    | KW_WORK
+    | KW_START
+    | KW_TRANSACTION
+    | KW_COMMIT
+    | KW_ROLLBACK
+    | KW_ONLY
+    | KW_WRITE
+    | KW_ISOLATION
+    | KW_LEVEL
+    | KW_SNAPSHOT
+    | KW_AUTOCOMMIT
+;
 
 //The following SQL2011 reserved keywords are used as cast function name only, it is a subset of the sql11ReservedKeywordsUsedAsIdentifier.
 sql11ReservedKeywordsUsedAsCastFunctionName

http://git-wip-us.apache.org/repos/asf/hive/blob/e57c3602/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index 0c191da..ad4efef 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -10042,6 +10042,25 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       viewsExpanded.add(createVwDesc.getViewName());
     }
 
+    switch(ast.getToken().getType()) {
+      case HiveParser.TOK_SET_AUTOCOMMIT:
+        assert ast.getChildCount() == 1;
+        if(ast.getChild(0).getType() == HiveParser.TOK_TRUE) {
+          setAutoCommitValue(true);
+        }
+        else if(ast.getChild(0).getType() == HiveParser.TOK_FALSE) {
+          setAutoCommitValue(false);
+        }
+        else {
+          assert false : "Unexpected child of TOK_SET_AUTOCOMMIT: " + ast.getChild(0).getType();
+        }
+        //fall through
+      case HiveParser.TOK_START_TRANSACTION:
+      case HiveParser.TOK_COMMIT:
+      case HiveParser.TOK_ROLLBACK:
+        SessionState.get().setCommandType(SemanticAnalyzerFactory.getOperation(ast.getToken().getType()));
+        return false;
+    }
     // 4. continue analyzing from the child ASTNode.
     Phase1Ctx ctx_1 = initPhase1Ctx();
     preProcessForInsert(child, qb);
@@ -10164,7 +10183,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     }
 
     // 6. Generate table access stats if required
-    if (HiveConf.getBoolVar(this.conf, HiveConf.ConfVars.HIVE_STATS_COLLECT_TABLEKEYS) == true) {
+    if (HiveConf.getBoolVar(this.conf, HiveConf.ConfVars.HIVE_STATS_COLLECT_TABLEKEYS)) {
       TableAccessAnalyzer tableAccessAnalyzer = new TableAccessAnalyzer(pCtx);
       setTableAccessInfo(tableAccessAnalyzer.analyzeTableAccess());
     }
@@ -10187,7 +10206,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     boolean isColumnInfoNeedForAuth = SessionState.get().isAuthorizationModeV2()
         && HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_AUTHORIZATION_ENABLED);
     if (isColumnInfoNeedForAuth
-        || HiveConf.getBoolVar(this.conf, HiveConf.ConfVars.HIVE_STATS_COLLECT_SCANCOLS) == true) {
+        || HiveConf.getBoolVar(this.conf, HiveConf.ConfVars.HIVE_STATS_COLLECT_SCANCOLS)) {
       ColumnAccessAnalyzer columnAccessAnalyzer = new ColumnAccessAnalyzer(pCtx);
       setColumnAccessInfo(columnAccessAnalyzer.analyzeColumnAccess());
     }
@@ -10657,7 +10676,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
    * Add default properties for table property. If a default parameter exists
    * in the tblProp, the value in tblProp will be kept.
    *
-   * @param table
+   * @param tblProp
    *          property map
    * @return Modified table property map
    */

http://git-wip-us.apache.org/repos/asf/hive/blob/e57c3602/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
index 2fdf1e7..a2fbc11 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
@@ -59,6 +59,7 @@ public final class SemanticAnalyzerFactory {
     commandType.put(HiveParser.TOK_ALTERTABLE_UNARCHIVE, HiveOperation.ALTERTABLE_UNARCHIVE);
     commandType.put(HiveParser.TOK_ALTERTABLE_PROPERTIES, HiveOperation.ALTERTABLE_PROPERTIES);
     commandType.put(HiveParser.TOK_ALTERTABLE_DROPPROPERTIES, HiveOperation.ALTERTABLE_PROPERTIES);
+    commandType.put(HiveParser.TOK_ALTERTABLE_EXCHANGEPARTITION, HiveOperation.ALTERTABLE_EXCHANGEPARTITION);
     commandType.put(HiveParser.TOK_SHOWDATABASES, HiveOperation.SHOWDATABASES);
     commandType.put(HiveParser.TOK_SHOWTABLES, HiveOperation.SHOWTABLES);
     commandType.put(HiveParser.TOK_SHOWCOLUMNS, HiveOperation.SHOWCOLUMNS);
@@ -111,6 +112,10 @@ public final class SemanticAnalyzerFactory {
     commandType.put(HiveParser.TOK_ALTERTABLE_PARTCOLTYPE, HiveOperation.ALTERTABLE_PARTCOLTYPE);
     commandType.put(HiveParser.TOK_SHOW_COMPACTIONS, HiveOperation.SHOW_COMPACTIONS);
     commandType.put(HiveParser.TOK_SHOW_TRANSACTIONS, HiveOperation.SHOW_TRANSACTIONS);
+    commandType.put(HiveParser.TOK_START_TRANSACTION, HiveOperation.START_TRANSACTION);
+    commandType.put(HiveParser.TOK_COMMIT, HiveOperation.COMMIT);
+    commandType.put(HiveParser.TOK_ROLLBACK, HiveOperation.ROLLBACK);
+    commandType.put(HiveParser.TOK_SET_AUTOCOMMIT, HiveOperation.SET_AUTOCOMMIT);
   }
 
   static {
@@ -270,6 +275,10 @@ public final class SemanticAnalyzerFactory {
       case HiveParser.TOK_DELETE_FROM:
         return new UpdateDeleteSemanticAnalyzer(conf);
 
+      case HiveParser.TOK_START_TRANSACTION:
+      case HiveParser.TOK_COMMIT:
+      case HiveParser.TOK_ROLLBACK:
+      case HiveParser.TOK_SET_AUTOCOMMIT:
       default: {
         SemanticAnalyzer semAnalyzer = HiveConf
             .getBoolVar(conf, HiveConf.ConfVars.HIVE_CBO_ENABLED) ? new CalcitePlanner(conf)
@@ -289,4 +298,7 @@ public final class SemanticAnalyzerFactory {
   private SemanticAnalyzerFactory() {
     // prevent instantiation
   }
+  static HiveOperation getOperation(int hiveParserToken) {
+    return commandType.get(hiveParserToken);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/e57c3602/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
index df37832..dee2136 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
@@ -100,7 +100,7 @@ public enum HiveOperation {
   CREATETABLE("CREATETABLE", null, new Privilege[]{Privilege.CREATE}),
   TRUNCATETABLE("TRUNCATETABLE", null, new Privilege[]{Privilege.DROP}),
   CREATETABLE_AS_SELECT("CREATETABLE_AS_SELECT", new Privilege[]{Privilege.SELECT}, new Privilege[]{Privilege.CREATE}),
-  QUERY("QUERY", new Privilege[]{Privilege.SELECT}, new Privilege[]{Privilege.ALTER_DATA, Privilege.CREATE}),
+  QUERY("QUERY", new Privilege[]{Privilege.SELECT}, new Privilege[]{Privilege.ALTER_DATA, Privilege.CREATE}, true, false),
   ALTERINDEX_PROPS("ALTERINDEX_PROPS",null, null),
   ALTERDATABASE("ALTERDATABASE", null, null),
   ALTERDATABASE_OWNER("ALTERDATABASE_OWNER", null, null),
@@ -111,11 +111,16 @@ public enum HiveOperation {
   ALTERTBLPART_SKEWED_LOCATION("ALTERTBLPART_SKEWED_LOCATION",
       new Privilege[] {Privilege.ALTER_DATA}, null),
   ALTERTABLE_PARTCOLTYPE("ALTERTABLE_PARTCOLTYPE", new Privilege[] { Privilege.SELECT }, new Privilege[] { Privilege.ALTER_DATA }),
+  ALTERTABLE_EXCHANGEPARTITION("ALTERTABLE_EXCHANGEPARTITION", null, null),
   ALTERVIEW_RENAME("ALTERVIEW_RENAME", new Privilege[] {Privilege.ALTER_METADATA}, null),
   ALTERVIEW_AS("ALTERVIEW_AS", new Privilege[] {Privilege.ALTER_METADATA}, null),
   ALTERTABLE_COMPACT("ALTERTABLE_COMPACT", new Privilege[]{Privilege.SELECT}, new Privilege[]{Privilege.ALTER_DATA}),
   SHOW_COMPACTIONS("SHOW COMPACTIONS", null, null),
-  SHOW_TRANSACTIONS("SHOW TRANSACTIONS", null, null);
+  SHOW_TRANSACTIONS("SHOW TRANSACTIONS", null, null),
+  START_TRANSACTION("START TRANSACTION", null, null, false, false),
+  COMMIT("COMMIT", null, null, true, true),
+  ROLLBACK("ROLLBACK", null, null, true, true),
+  SET_AUTOCOMMIT("SET AUTOCOMMIT", null, null, true, false);
   ;
 
   private String operationName;
@@ -124,6 +129,12 @@ public enum HiveOperation {
 
   private Privilege[] outputRequiredPrivileges;
 
+  /**
+   * Only a small set of operations is allowed inside an open transactions, e.g. DML
+   */
+  private final boolean allowedInTransaction;
+  private final boolean requiresOpenTransaction;
+
   public Privilege[] getInputRequiredPrivileges() {
     return inputRequiredPrivileges;
   }
@@ -136,11 +147,26 @@ public enum HiveOperation {
     return operationName;
   }
 
+  public boolean isAllowedInTransaction() {
+    return allowedInTransaction;
+  }
+  public boolean isRequiresOpenTransaction() { return requiresOpenTransaction; }
+
   private HiveOperation(String operationName,
-      Privilege[] inputRequiredPrivileges, Privilege[] outputRequiredPrivileges) {
+                        Privilege[] inputRequiredPrivileges, Privilege[] outputRequiredPrivileges) {
+    this(operationName, inputRequiredPrivileges, outputRequiredPrivileges, false, false);
+  }
+  private HiveOperation(String operationName,
+      Privilege[] inputRequiredPrivileges, Privilege[] outputRequiredPrivileges,
+      boolean allowedInTransaction, boolean requiresOpenTransaction) {
     this.operationName = operationName;
     this.inputRequiredPrivileges = inputRequiredPrivileges;
     this.outputRequiredPrivileges = outputRequiredPrivileges;
+    this.requiresOpenTransaction = requiresOpenTransaction;
+    if(requiresOpenTransaction) {
+      allowedInTransaction = true;
+    }
+    this.allowedInTransaction = allowedInTransaction;
   }
 
   public static class PrivilegeAgreement {

http://git-wip-us.apache.org/repos/asf/hive/blob/e57c3602/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandProcessorResponse.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandProcessorResponse.java b/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandProcessorResponse.java
index 4584517..21b7457 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandProcessorResponse.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandProcessorResponse.java
@@ -19,16 +19,19 @@
 package org.apache.hadoop.hive.ql.processors;
 
 import org.apache.hadoop.hive.metastore.api.Schema;
+import org.apache.hadoop.hive.ql.ErrorMsg;
 
 /**
  * Encapsulates the basic response info returned by classes the implement the
  * <code>CommandProcessor</code> interface. Typically <code>errorMessage</code>
  * and <code>SQLState</code> will only be set if the <code>responseCode</code>
- * is not 0.
+ * is not 0.  Note that often {@code responseCode} ends up the exit value of
+ * command shell process so should keep it to < 127.
  */
 public class CommandProcessorResponse {
   private final int responseCode;
   private final String errorMessage;
+  private final int hiveErrorCode;
   private final String SQLState;
   private final Schema resSchema;
 
@@ -49,6 +52,10 @@ public class CommandProcessorResponse {
   public CommandProcessorResponse(int responseCode, String errorMessage, String SQLState, Schema schema) {
     this(responseCode, errorMessage, SQLState, schema, null);
   }
+  public CommandProcessorResponse(int responseCode, ErrorMsg canonicalErrMsg, Throwable t, String ... msgArgs) {
+    this(responseCode, canonicalErrMsg.format(msgArgs),
+      canonicalErrMsg.getSQLState(), null, t, canonicalErrMsg.getErrorCode());
+  }
 
   /**
    * Create CommandProcessorResponse object indicating an error.
@@ -63,12 +70,17 @@ public class CommandProcessorResponse {
   }
 
   public CommandProcessorResponse(int responseCode, String errorMessage, String SQLState,
-      Schema schema, Throwable exception) {
+                                  Schema schema, Throwable exception) {
+    this(responseCode, errorMessage, SQLState, schema, exception, -1);
+  }
+  public CommandProcessorResponse(int responseCode, String errorMessage, String SQLState,
+      Schema schema, Throwable exception, int hiveErrorCode) {
     this.responseCode = responseCode;
     this.errorMessage = errorMessage;
     this.SQLState = SQLState;
     this.resSchema = schema;
     this.exception = exception;
+    this.hiveErrorCode = hiveErrorCode;
   }
 
   public int getResponseCode() { return responseCode; }
@@ -76,8 +88,11 @@ public class CommandProcessorResponse {
   public String getSQLState() { return SQLState; }
   public Schema getSchema() { return resSchema; }
   public Throwable getException() { return exception; }
+  public int getErrorCode() { return hiveErrorCode; }
   public String toString() {
-    return "(" + responseCode + "," + errorMessage + "," + SQLState + 
+    return "(" + responseCode + "," + errorMessage + "," +
+      (hiveErrorCode > 0 ? hiveErrorCode + "," : "" ) +
+      SQLState +
       (resSchema == null ? "" : ",") +
       (exception == null ? "" : exception.getMessage()) + ")";
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/e57c3602/ql/src/java/org/apache/hadoop/hive/ql/processors/HiveCommand.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/processors/HiveCommand.java b/ql/src/java/org/apache/hadoop/hive/ql/processors/HiveCommand.java
index 319a79b..c8c9831 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/processors/HiveCommand.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/processors/HiveCommand.java
@@ -75,6 +75,9 @@ public enum HiveCommand {
       } else if(command.length > 1 && "from".equalsIgnoreCase(command[1])) {
         //special handling for SQL "delete from <table> where..."
         return null;
+      }
+      else if(command.length > 1 && "set".equalsIgnoreCase(command[0]) && "autocommit".equalsIgnoreCase(command[1])) {
+        return null;//don't want set autocommit true|false to get mixed with set hive.foo.bar...
       } else if (COMMANDS.contains(cmd)) {
         HiveCommand hiveCommand = HiveCommand.valueOf(cmd);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/e57c3602/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
index b974b59..71be469 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
@@ -125,6 +125,11 @@ public enum HiveOperationType {
   ADD,
   DELETE,
   COMPILE,
+  START_TRANSACTION,
+  COMMIT,
+  ROLLBACK,
+  SET_AUTOCOMMIT,
+  ALTERTABLE_EXCHANGEPARTITION,
   // ==== Hive command operations ends here ==== //
 
   // ==== HiveServer2 metadata api types start here ==== //

http://git-wip-us.apache.org/repos/asf/hive/blob/e57c3602/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
index a6226b6..8e61d57 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
@@ -400,6 +400,17 @@ public class Operation2Privilege {
     op2Priv.put(HiveOperationType.GET_COLUMNS,
         PrivRequirement.newIOPrivRequirement(SEL_NOGRANT_AR, null));
 
+    op2Priv.put(HiveOperationType.START_TRANSACTION, PrivRequirement.newIOPrivRequirement
+      (null, null));
+    op2Priv.put(HiveOperationType.COMMIT, PrivRequirement.newIOPrivRequirement
+      (null, null));
+    op2Priv.put(HiveOperationType.ROLLBACK, PrivRequirement.newIOPrivRequirement
+      (null, null));
+    op2Priv.put(HiveOperationType.SET_AUTOCOMMIT, PrivRequirement.newIOPrivRequirement
+      (null, null));
+    op2Priv.put(HiveOperationType.ALTERTABLE_EXCHANGEPARTITION,
+      PrivRequirement.newIOPrivRequirement(null, null));
+
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/e57c3602/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
index 49d64db..510d8a7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
@@ -116,7 +116,7 @@ public class SessionState {
   /**
    * current configuration.
    */
-  protected HiveConf conf;
+  private final HiveConf conf;
 
   /**
    * silent mode.
@@ -249,23 +249,6 @@ public class SessionState {
   private HiveTxnManager txnMgr = null;
 
   /**
-   * When {@link #setCurrentTxn(long)} is set to this or {@link #getCurrentTxn()}} returns this it
-   * indicates that there is not a current transaction in this session.
-  */
-  public static final long NO_CURRENT_TXN = -1L;
-
-  /**
-   * Transaction currently open
-   */
-  private long currentTxn = NO_CURRENT_TXN;
-
-  /**
-   * Whether we are in auto-commit state or not.  Currently we are always in auto-commit,
-   * so there are not setters for this yet.
-   */
-  private final boolean txnAutoCommit = true;
-
-  /**
    * store the jars loaded last time
    */
   private final Set<String> preReloadableAuxJars = new HashSet<String>();
@@ -293,9 +276,6 @@ public class SessionState {
     return conf;
   }
 
-  public void setConf(HiveConf conf) {
-    this.conf = conf;
-  }
 
   public File getTmpOutputFile() {
     return tmpOutputFile;
@@ -410,18 +390,6 @@ public class SessionState {
     return txnMgr;
   }
 
-  public long getCurrentTxn() {
-    return currentTxn;
-  }
-
-  public void setCurrentTxn(long currTxn) {
-    currentTxn = currTxn;
-  }
-
-  public boolean isAutoCommit() {
-    return txnAutoCommit;
-  }
-
   public HadoopShims.HdfsEncryptionShim getHdfsEncryptionShim() throws HiveException {
     if (hdfsEncryptionShim == null) {
       try {

http://git-wip-us.apache.org/repos/asf/hive/blob/e57c3602/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java
new file mode 100644
index 0000000..c73621f
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java
@@ -0,0 +1,473 @@
+package org.apache.hadoop.hive.ql;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.txn.TxnDbUtil;
+import org.apache.hadoop.hive.ql.io.AcidUtils;
+import org.apache.hadoop.hive.ql.io.orc.FileDump;
+import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+
+/**
+ * The LockManager is not ready, but for no-concurrency straight-line path we can 
+ * test AC=true, and AC=false with commit/rollback/exception and test resulting data.
+ * 
+ * Can also test, calling commit in AC=true mode, etc, toggling AC...
+ */
+public class TestTxnCommands {
+  private static final String TEST_DATA_DIR = new File(System.getProperty("java.io.tmpdir") +
+    File.separator + TestTxnCommands.class.getCanonicalName()
+    + "-" + System.currentTimeMillis()
+  ).getPath().replaceAll("\\\\", "/");
+  private static final String TEST_WAREHOUSE_DIR = TEST_DATA_DIR + "/warehouse";
+  //bucket count for test tables; set it to 1 for easier debugging
+  private static int BUCKET_COUNT = 2;
+  @Rule
+  public TestName testName = new TestName();
+  private HiveConf hiveConf;
+  private Driver d;
+  private static enum Table {
+    ACIDTBL("acidTbl"),
+    ACIDTBL2("acidTbl2"),
+    NONACIDORCTBL("nonAcidOrcTbl"),
+    NONACIDORCTBL2("nonAcidOrcTbl2");
+    
+    private final String name;
+    @Override
+    public String toString() {
+      return name;
+    }
+    Table(String name) {
+      this.name = name;
+    }
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    tearDown();
+    hiveConf = new HiveConf(this.getClass());
+    hiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, "");
+    hiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, "");
+    hiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false");
+    hiveConf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, TEST_WAREHOUSE_DIR);
+    TxnDbUtil.setConfValues(hiveConf);
+    hiveConf.setBoolVar(HiveConf.ConfVars.HIVEENFORCEBUCKETING, true);
+    TxnDbUtil.prepDb();
+    File f = new File(TEST_WAREHOUSE_DIR);
+    if (f.exists()) {
+      FileUtil.fullyDelete(f);
+    }
+    if (!(new File(TEST_WAREHOUSE_DIR).mkdirs())) {
+      throw new RuntimeException("Could not create " + TEST_WAREHOUSE_DIR);
+    }
+    SessionState.start(new SessionState(hiveConf));
+    d = new Driver(hiveConf);
+    dropTables();
+    runStatementOnDriver("create table " + Table.ACIDTBL + "(a int, b int) clustered by (a) into " + BUCKET_COUNT + " buckets stored as orc TBLPROPERTIES ('transactional'='true')");
+    runStatementOnDriver("create table " + Table.NONACIDORCTBL + "(a int, b int) clustered by (a) into " + BUCKET_COUNT + " buckets stored as orc TBLPROPERTIES ('transactional'='false')");
+    runStatementOnDriver("create table " + Table.NONACIDORCTBL2 + "(a int, b int) clustered by (a) into " + BUCKET_COUNT + " buckets stored as orc TBLPROPERTIES ('transactional'='false')");
+    runStatementOnDriver("create temporary  table " + Table.ACIDTBL2 + "(a int, b int, c int) clustered by (c) into " + BUCKET_COUNT + " buckets stored as orc TBLPROPERTIES ('transactional'='true')");
+  }
+  private void dropTables() throws Exception {
+    for(Table t : Table.values()) {
+      runStatementOnDriver("drop table if exists " + t);
+    }
+  }
+  @After
+  public void tearDown() throws Exception {
+    try {
+      if (d != null) {
+        runStatementOnDriver("set autocommit true");
+        dropTables();
+        d.destroy();
+        d.close();
+        d = null;
+      }
+    } finally {
+      TxnDbUtil.cleanDb();
+      FileUtils.deleteDirectory(new File(TEST_DATA_DIR));
+    }
+  }
+  @Test 
+  public void testInsertOverwrite() throws Exception {
+    runStatementOnDriver("insert overwrite table " + Table.NONACIDORCTBL + " select a,b from " + Table.NONACIDORCTBL2);
+    runStatementOnDriver("create table " + Table.NONACIDORCTBL2 + "3(a int, b int) clustered by (a) into " + BUCKET_COUNT + " buckets stored as orc TBLPROPERTIES ('transactional'='false')");
+
+  }
+  @Ignore("not needed but useful for testing")
+  @Test
+  public void testNonAcidInsert() throws Exception {
+    runStatementOnDriver("insert into " + Table.NONACIDORCTBL + "(a,b) values(1,2)");
+    List<String> rs = runStatementOnDriver("select a,b from " + Table.NONACIDORCTBL);
+    runStatementOnDriver("insert into " + Table.NONACIDORCTBL + "(a,b) values(2,3)");
+    List<String> rs1 = runStatementOnDriver("select a,b from " + Table.NONACIDORCTBL);
+  }
+
+  /**
+   * Useful for debugging.  Dumps ORC file in JSON to CWD.
+   */
+  private void dumpBucketData(Table table, long txnId, int stmtId, int bucketNum) throws Exception {
+    if(true) {
+      return;
+    }
+    Path bucket = AcidUtils.createBucketFile(new Path(new Path(TEST_WAREHOUSE_DIR, table.toString().toLowerCase()), AcidUtils.deltaSubdir(txnId, txnId, stmtId)), bucketNum);
+    FileOutputStream delta = new FileOutputStream(testName.getMethodName() + "_" + bucket.getParent().getName() + "_" +  bucket.getName());
+//    try {
+//      FileDump.printJsonData(hiveConf, bucket.toString(), delta);
+//    }
+//    catch(FileNotFoundException ex) {
+      ;//this happens if you change BUCKET_COUNT
+//    }
+    delta.close();
+  }
+  /**
+   * Dump all data in the table by bucket in JSON format
+   */
+  private void dumpTableData(Table table, long txnId, int stmtId) throws Exception {
+    for(int bucketNum = 0; bucketNum < BUCKET_COUNT; bucketNum++) {
+      dumpBucketData(table, txnId, stmtId, bucketNum);
+    }
+  }
+  @Test
+  public void testSimpleAcidInsert() throws Exception {
+    int[][] rows1 = {{1,2},{3,4}};
+    runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) " + makeValuesClause(rows1));
+    //List<String> rs = runStatementOnDriver("select a,b from " + Table.ACIDTBL + " order by a,b");
+    //Assert.assertEquals("Data didn't match in autocommit=true (rs)", stringifyValues(rows1), rs);
+    runStatementOnDriver("set autocommit false");
+    runStatementOnDriver("START TRANSACTION");
+    int[][] rows2 = {{5,6},{7,8}};
+    runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) " + makeValuesClause(rows2));
+    List<String> allData = stringifyValues(rows1);
+    allData.addAll(stringifyValues(rows2));
+    List<String> rs0 = runStatementOnDriver("select a,b from " + Table.ACIDTBL + " order by a,b");
+    Assert.assertEquals("Data didn't match inside tx (rs0)", allData, rs0);
+    runStatementOnDriver("COMMIT WORK");
+    dumpTableData(Table.ACIDTBL, 1, 0);
+    dumpTableData(Table.ACIDTBL, 2, 0);
+    runStatementOnDriver("select a,b from " + Table.ACIDTBL + " order by a,b");
+    runStatementOnDriver("COMMIT");//txn started implicitly by previous statement
+    runStatementOnDriver("set autocommit true");
+    List<String> rs1 = runStatementOnDriver("select a,b from " + Table.ACIDTBL + " order by a,b");
+    Assert.assertEquals("Data didn't match inside tx (rs0)", allData, rs1);
+  }
+
+  /**
+   * add tests for all transitions - AC=t, AC=t, AC=f, commit (for example)
+   * @throws Exception
+   */
+  @Test
+  public void testErrors() throws Exception {
+    runStatementOnDriver("set autocommit true");
+    CommandProcessorResponse cpr = runStatementOnDriverNegative("start transaction");
+    Assert.assertEquals("Error didn't match: " + cpr, ErrorMsg.OP_NOT_ALLOWED_IN_AUTOCOMMIT.getErrorCode(), cpr.getErrorCode());
+    runStatementOnDriver("set autocommit false");
+    runStatementOnDriver("start transaction");
+    CommandProcessorResponse cpr2 = runStatementOnDriverNegative("create table foo(x int, y int)");
+    Assert.assertEquals("Expected DDL to fail in an open txn", ErrorMsg.OP_NOT_ALLOWED_IN_TXN.getErrorCode(), cpr2.getErrorCode());
+    runStatementOnDriver("set autocommit true");
+    CommandProcessorResponse cpr3 = runStatementOnDriverNegative("update " + Table.ACIDTBL + " set a = 1 where b != 1");
+    Assert.assertEquals("Expected update of bucket column to fail",
+      "FAILED: SemanticException [Error 10302]: Updating values of bucketing columns is not supported.  Column a.",
+      cpr3.getErrorMessage());
+    //line below should in principle work but Driver doesn't propagate errorCode properly
+    //Assert.assertEquals("Expected update of bucket column to fail", ErrorMsg.UPDATE_CANNOT_UPDATE_BUCKET_VALUE.getErrorCode(), cpr3.getErrorCode());
+    cpr3 = runStatementOnDriverNegative("commit work");//not allowed in AC=true
+    Assert.assertEquals("Error didn't match: " + cpr3, ErrorMsg.OP_NOT_ALLOWED_IN_AUTOCOMMIT.getErrorCode(), cpr.getErrorCode());
+    cpr3 = runStatementOnDriverNegative("rollback work");//not allowed in AC=true
+    Assert.assertEquals("Error didn't match: " + cpr3, ErrorMsg.OP_NOT_ALLOWED_IN_AUTOCOMMIT.getErrorCode(), cpr.getErrorCode());
+    runStatementOnDriver("set autocommit false");
+    cpr3 = runStatementOnDriverNegative("commit");//not allowed in w/o tx
+    Assert.assertEquals("Error didn't match: " + cpr3, ErrorMsg.OP_NOT_ALLOWED_IN_AUTOCOMMIT.getErrorCode(), cpr.getErrorCode());
+    cpr3 = runStatementOnDriverNegative("rollback");//not allowed in w/o tx
+    Assert.assertEquals("Error didn't match: " + cpr3, ErrorMsg.OP_NOT_ALLOWED_IN_AUTOCOMMIT.getErrorCode(), cpr.getErrorCode());
+    runStatementOnDriver("start transaction");
+    cpr3 = runStatementOnDriverNegative("start transaction");//not allowed in a tx
+    Assert.assertEquals("Expected start transaction to fail", ErrorMsg.OP_NOT_ALLOWED_IN_TXN.getErrorCode(), cpr3.getErrorCode());
+    runStatementOnDriver("start transaction");//ok since previously opened txn was killed
+    runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) values(1,2)");
+    List<String> rs0 = runStatementOnDriver("select a,b from " + Table.ACIDTBL + " order by a,b");
+    Assert.assertEquals("Can't see my own write", 1, rs0.size());
+    runStatementOnDriver("set autocommit true");//this should commit previous txn
+    rs0 = runStatementOnDriver("select a,b from " + Table.ACIDTBL + " order by a,b");
+    Assert.assertEquals("Can't see my own write", 1, rs0.size());
+  }
+  @Test 
+  public void testReadMyOwnInsert() throws Exception {
+    runStatementOnDriver("set autocommit false");
+    runStatementOnDriver("START TRANSACTION");
+    List<String> rs = runStatementOnDriver("select * from " + Table.ACIDTBL);
+    Assert.assertEquals("Expected empty " + Table.ACIDTBL, 0, rs.size());
+    runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) values(1,2)");
+    List<String> rs0 = runStatementOnDriver("select a,b from " + Table.ACIDTBL + " order by a,b");
+    Assert.assertEquals("Can't see my own write", 1, rs0.size());
+    runStatementOnDriver("commit");
+    runStatementOnDriver("START TRANSACTION");
+    List<String> rs1 = runStatementOnDriver("select a,b from " + Table.ACIDTBL + " order by a,b");
+    runStatementOnDriver("rollback work");
+    Assert.assertEquals("Can't see write after commit", 1, rs1.size());
+  }
+  @Test
+  public void testImplicitRollback() throws Exception {
+    runStatementOnDriver("set autocommit false");
+    runStatementOnDriver("START TRANSACTION");
+    runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) values(1,2)");
+    List<String> rs0 = runStatementOnDriver("select a,b from " + Table.ACIDTBL + " order by a,b");
+    Assert.assertEquals("Can't see my own write", 1, rs0.size());
+    //next command should produce an error
+    CommandProcessorResponse cpr = runStatementOnDriverNegative("select * from no_such_table");
+    Assert.assertEquals("Txn didn't fail?",
+      "FAILED: SemanticException [Error 10001]: Line 1:14 Table not found 'no_such_table'",
+      cpr.getErrorMessage());
+    runStatementOnDriver("start transaction");
+    List<String> rs1 = runStatementOnDriver("select a,b from " + Table.ACIDTBL + " order by a,b");
+    runStatementOnDriver("commit");
+    Assert.assertEquals("Didn't rollback as expected", 0, rs1.size());
+  }
+  @Test
+  public void testExplicitRollback() throws Exception {
+    runStatementOnDriver("set autocommit false");
+    runStatementOnDriver("START TRANSACTION");
+    runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) values(1,2)");
+    runStatementOnDriver("ROLLBACK");
+    runStatementOnDriver("set autocommit true");
+    List<String> rs = runStatementOnDriver("select a,b from " + Table.ACIDTBL + " order by a,b");
+    Assert.assertEquals("Rollback didn't rollback", 0, rs.size());
+  }
+
+  @Test
+  public void testMultipleInserts() throws Exception {
+    runStatementOnDriver("set autocommit false");
+    runStatementOnDriver("START TRANSACTION");
+    int[][] rows1 = {{1,2},{3,4}};
+    runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) " + makeValuesClause(rows1));
+    int[][] rows2 = {{5,6},{7,8}};
+    runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) " + makeValuesClause(rows2));
+    List<String> allData = stringifyValues(rows1);
+    allData.addAll(stringifyValues(rows2));
+    List<String> rs = runStatementOnDriver("select a,b from " + Table.ACIDTBL + " order by a,b");
+    Assert.assertEquals("Content didn't match before commit rs", allData, rs);
+    runStatementOnDriver("commit");
+    dumpTableData(Table.ACIDTBL, 1, 0);
+    dumpTableData(Table.ACIDTBL, 1, 1);
+    runStatementOnDriver("set autocommit true");
+    List<String> rs1 = runStatementOnDriver("select a,b from " + Table.ACIDTBL + " order by a,b");
+    Assert.assertEquals("Content didn't match after commit rs1", allData, rs1);
+  }
+  @Test
+  public void testDelete() throws Exception {
+    int[][] rows1 = {{1,2},{3,4}};
+    runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) " + makeValuesClause(rows1));
+    List<String> rs0 = runStatementOnDriver("select a,b from " + Table.ACIDTBL + " order by a,b");
+    Assert.assertEquals("Content didn't match rs0", stringifyValues(rows1), rs0);
+    runStatementOnDriver("set autocommit false");
+    runStatementOnDriver("START TRANSACTION");
+    runStatementOnDriver("delete from " + Table.ACIDTBL + " where b = 4");
+    int[][] updatedData2 = {{1,2}};
+    List<String> rs3 = runStatementOnDriver("select a,b from " + Table.ACIDTBL + " order by a,b");
+    Assert.assertEquals("Wrong data after delete", stringifyValues(updatedData2), rs3);
+    runStatementOnDriver("commit");
+    runStatementOnDriver("set autocommit true");
+    List<String> rs4 = runStatementOnDriver("select a,b from " + Table.ACIDTBL + " order by a,b");
+    Assert.assertEquals("Wrong data after commit", stringifyValues(updatedData2), rs4);
+  }
+
+  @Test
+  public void testUpdateOfInserts() throws Exception {
+    int[][] rows1 = {{1,2},{3,4}};
+    runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) " + makeValuesClause(rows1));
+    List<String> rs0 = runStatementOnDriver("select a,b from " + Table.ACIDTBL + " order by a,b");
+    Assert.assertEquals("Content didn't match rs0", stringifyValues(rows1), rs0);
+    runStatementOnDriver("set autocommit false");
+    runStatementOnDriver("START TRANSACTION");
+    int[][] rows2 = {{5,6},{7,8}};
+    runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) " + makeValuesClause(rows2));
+    List<String> rs1 = runStatementOnDriver("select a,b from " + Table.ACIDTBL + " order by a,b");
+    List<String> allData = stringifyValues(rows1);
+    allData.addAll(stringifyValues(rows2));
+    Assert.assertEquals("Content didn't match rs1", allData, rs1);
+    runStatementOnDriver("update " + Table.ACIDTBL + " set b = 1 where b != 1");
+    int[][] updatedData = {{1,1},{3,1},{5,1},{7,1}};
+    List<String> rs2 = runStatementOnDriver("select a,b from " + Table.ACIDTBL + " order by a,b");
+    Assert.assertEquals("Wrong data after update", stringifyValues(updatedData), rs2);
+    runStatementOnDriver("commit");
+    runStatementOnDriver("set autocommit true");
+    List<String> rs4 = runStatementOnDriver("select a,b from " + Table.ACIDTBL + " order by a,b");
+    Assert.assertEquals("Wrong data after commit", stringifyValues(updatedData), rs4);
+  }
+  @Test
+  public void testUpdateDeleteOfInserts() throws Exception {
+    int[][] rows1 = {{1,2},{3,4}};
+    runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) " + makeValuesClause(rows1));
+    List<String> rs0 = runStatementOnDriver("select a,b from " + Table.ACIDTBL + " order by a,b");
+    Assert.assertEquals("Content didn't match rs0", stringifyValues(rows1), rs0);
+    runStatementOnDriver("set autocommit false");
+    runStatementOnDriver("START TRANSACTION");
+    int[][] rows2 = {{5,6},{7,8}};
+    runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) " + makeValuesClause(rows2));
+    List<String> rs1 = runStatementOnDriver("select a,b from " + Table.ACIDTBL + " order by a,b");
+    List<String> allData = stringifyValues(rows1);
+    allData.addAll(stringifyValues(rows2));
+    Assert.assertEquals("Content didn't match rs1", allData, rs1);
+    runStatementOnDriver("update " + Table.ACIDTBL + " set b = 1 where b != 1");
+    int[][] updatedData = {{1,1},{3,1},{5,1},{7,1}};
+    List<String> rs2 = runStatementOnDriver("select a,b from " + Table.ACIDTBL + " order by a,b");
+    Assert.assertEquals("Wrong data after update", stringifyValues(updatedData), rs2);
+    runStatementOnDriver("delete from " + Table.ACIDTBL + " where a = 7 and b = 1");
+    dumpTableData(Table.ACIDTBL, 1, 0);
+    dumpTableData(Table.ACIDTBL, 2, 0);
+    dumpTableData(Table.ACIDTBL, 2, 2);
+    dumpTableData(Table.ACIDTBL, 2, 4);
+    int[][] updatedData2 = {{1,1},{3,1},{5,1}};
+    List<String> rs3 = runStatementOnDriver("select a,b from " + Table.ACIDTBL + " order by a,b");
+    Assert.assertEquals("Wrong data after delete", stringifyValues(updatedData2), rs3);
+    runStatementOnDriver("commit");
+    runStatementOnDriver("set autocommit true");
+    List<String> rs4 = runStatementOnDriver("select a,b from " + Table.ACIDTBL + " order by a,b");
+    Assert.assertEquals("Wrong data after commit", stringifyValues(updatedData2), rs4);
+  }
+  @Test
+  public void testMultipleDelete() throws Exception {
+    int[][] rows1 = {{1,2},{3,4},{5,6},{7,8}};
+    runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) " + makeValuesClause(rows1));
+    List<String> rs0 = runStatementOnDriver("select a,b from " + Table.ACIDTBL + " order by a,b");
+    Assert.assertEquals("Content didn't match rs0", stringifyValues(rows1), rs0);
+    runStatementOnDriver("set autocommit false");
+    runStatementOnDriver("START TRANSACTION");
+    runStatementOnDriver("delete from " + Table.ACIDTBL + " where b = 8");
+    int[][] updatedData2 = {{1,2},{3,4},{5,6}};
+    List<String> rs2 = runStatementOnDriver("select a,b from " + Table.ACIDTBL + " order by a,b");
+    Assert.assertEquals("Wrong data after delete", stringifyValues(updatedData2), rs2);
+    runStatementOnDriver("delete from " + Table.ACIDTBL + " where b = 4");
+    int[][] updatedData3 = {{1, 2}, {5, 6}};
+    List<String> rs3 = runStatementOnDriver("select a,b from " + Table.ACIDTBL + " order by a,b");
+    Assert.assertEquals("Wrong data after delete2", stringifyValues(updatedData3), rs3);
+    runStatementOnDriver("update " + Table.ACIDTBL + " set b=3");
+    dumpTableData(Table.ACIDTBL, 1, 0);
+    //nothing actually hashes to bucket0, so update/delete deltas don't have it
+    dumpTableData(Table.ACIDTBL, 2, 0);
+    dumpTableData(Table.ACIDTBL, 2, 2);
+    dumpTableData(Table.ACIDTBL, 2, 4);
+    List<String> rs5 = runStatementOnDriver("select a,b from " + Table.ACIDTBL + " order by a,b");
+    int [][] updatedData4 = {{1,3},{5,3}};
+    Assert.assertEquals("Wrong data after delete", stringifyValues(updatedData4), rs5);
+    runStatementOnDriver("commit");
+    runStatementOnDriver("set autocommit true");
+    List<String> rs4 = runStatementOnDriver("select a,b from " + Table.ACIDTBL + " order by a,b");
+    Assert.assertEquals("Wrong data after commit", stringifyValues(updatedData4), rs4);
+  }
+  @Test
+  public void testDeleteIn() throws Exception {
+    runStatementOnDriver("delete from " + Table.ACIDTBL + " where a IN (SELECT A.a from " +
+      Table.ACIDTBL + "  A)");
+    int[][] tableData = {{1,2},{3,2},{5,2},{1,3},{3,3},{5,3}};
+    runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) " + makeValuesClause(tableData));
+    runStatementOnDriver("insert into " + Table.ACIDTBL2 + "(a,b,c) values(1,7,17),(3,7,17)");
+//    runStatementOnDriver("select b from " + Table.ACIDTBL + " where a in (select b from " + Table.NONACIDORCTBL + ")");
+    runStatementOnDriver("delete from " + Table.ACIDTBL + " where a in(select a from " + Table.ACIDTBL2 + ")");
+//    runStatementOnDriver("delete from " + Table.ACIDTBL + " where a in(select a from " + Table.NONACIDORCTBL + ")");
+    runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) select a,b from " + Table.ACIDTBL2);
+    List<String> rs = runStatementOnDriver("select a,b from " + Table.ACIDTBL + " order by a,b");
+    int[][] updatedData = {{1,7},{3,7},{5,2},{5,3}};
+    Assert.assertEquals("Bulk update failed", stringifyValues(updatedData), rs);
+  }
+
+  /**
+   * takes raw data and turns it into a string as if from Driver.getResults()
+   * sorts rows in dictionary order
+   */
+  private List<String> stringifyValues(int[][] rowsIn) {
+    assert rowsIn.length > 0;
+    int[][] rows = rowsIn.clone();
+    Arrays.sort(rows, new RowComp());
+    List<String> rs = new ArrayList<String>();
+    for(int[] row : rows) {
+      assert row.length > 0;
+      StringBuilder sb = new StringBuilder();
+      for(int value : row) {
+        sb.append(value).append("\t");
+      }
+      sb.setLength(sb.length() - 1);
+      rs.add(sb.toString());
+    }
+    return rs;
+  }
+  private static final class RowComp implements Comparator<int[]> {
+    public int compare(int[] row1, int[] row2) {
+      assert row1 != null && row2 != null && row1.length == row2.length;
+      for(int i = 0; i < row1.length; i++) {
+        int comp = Integer.compare(row1[i], row2[i]);
+        if(comp != 0) {
+          return comp;
+        }
+      }
+      return 0;
+    }
+  }
+  private String makeValuesClause(int[][] rows) {
+    assert rows.length > 0;
+    StringBuilder sb = new StringBuilder("values");
+    for(int[] row : rows) {
+      assert row.length > 0;
+      if(row.length > 1) {
+        sb.append("(");
+      }
+      for(int value : row) {
+        sb.append(value).append(",");
+      }
+      sb.setLength(sb.length() - 1);//remove trailing comma
+      if(row.length > 1) {
+        sb.append(")");
+      }
+      sb.append(",");
+    }
+    sb.setLength(sb.length() - 1);//remove trailing comma
+    return sb.toString();
+  }
+  
+  private List<String> runStatementOnDriver(String stmt) throws Exception {
+    CommandProcessorResponse cpr = d.run(stmt);
+    if(cpr.getResponseCode() != 0) {
+      throw new RuntimeException(stmt + " failed: " + cpr);
+    }
+    List<String> rs = new ArrayList<String>();
+    d.getResults(rs);
+    return rs;
+  }
+  private CommandProcessorResponse runStatementOnDriverNegative(String stmt) throws Exception {
+    CommandProcessorResponse cpr = d.run(stmt);
+    if(cpr.getResponseCode() != 0) {
+      return cpr;
+    }
+    throw new RuntimeException("Didn't get expected failure!");
+  }
+
+//  @Ignore
+  @Test
+  public void exchangePartition() throws Exception {
+    runStatementOnDriver("create database ex1");
+    runStatementOnDriver("create database ex2");
+
+    runStatementOnDriver("CREATE TABLE ex1.exchange_part_test1 (f1 string) PARTITIONED BY (ds STRING)");
+    runStatementOnDriver("CREATE TABLE ex2.exchange_part_test2 (f1 string) PARTITIONED BY (ds STRING)");
+    runStatementOnDriver("ALTER TABLE ex2.exchange_part_test2 ADD PARTITION (ds='2013-04-05')");
+    runStatementOnDriver("ALTER TABLE ex1.exchange_part_test1 EXCHANGE PARTITION (ds='2013-04-05') WITH TABLE ex2.exchange_part_test2");
+  }
+}


[15/43] hive git commit: HIVE-11077 Add support in parser and wire up to txn manager (Eugene Koifman, reviewed by Alan Gates)

Posted by xu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/e57c3602/ql/src/test/org/apache/hadoop/hive/ql/parse/positive/TestTransactionStatement.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/positive/TestTransactionStatement.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/positive/TestTransactionStatement.java
new file mode 100644
index 0000000..b7f8263
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/positive/TestTransactionStatement.java
@@ -0,0 +1,102 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.parse.positive;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.ParseDriver;
+import org.apache.hadoop.hive.ql.parse.ParseException;
+import org.apache.hadoop.hive.ql.parse.SemanticAnalyzer;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+
+/**
+ * Basic parser tests for multi-statement transactions
+ */
+public class TestTransactionStatement {
+  private static SessionState sessionState;
+  private ParseDriver pd;
+
+  @BeforeClass
+  public static void initialize() {
+    HiveConf conf = new HiveConf(SemanticAnalyzer.class);
+    sessionState = SessionState.start(conf);
+  }
+  @AfterClass
+  public static void cleanUp() throws IOException {
+    if(sessionState != null) {
+      sessionState.close();
+    }
+  }
+
+  @Before
+  public void setup() throws SemanticException {
+    pd = new ParseDriver();
+  }
+
+  ASTNode parse(String query) throws ParseException {
+    ASTNode nd = pd.parse(query);
+    return (ASTNode) nd.getChild(0);
+  }
+  @Test
+  public void testTxnStart() throws ParseException {
+    ASTNode ast = parse("START TRANSACTION");
+    Assert.assertEquals("AST doesn't match",
+      "TOK_START_TRANSACTION", ast.toStringTree());
+    
+    ast = parse("START TRANSACTION ISOLATION LEVEL SNAPSHOT");
+    Assert.assertEquals("AST doesn't match",
+      "(TOK_START_TRANSACTION (TOK_ISOLATION_LEVEL TOK_ISOLATION_SNAPSHOT))", ast.toStringTree());
+    
+    ast = parse("START TRANSACTION READ ONLY");
+    Assert.assertEquals("AST doesn't match",
+      "(TOK_START_TRANSACTION (TOK_TXN_ACCESS_MODE TOK_TXN_READ_ONLY))", ast.toStringTree());
+    
+    ast = parse("START TRANSACTION READ WRITE, ISOLATION LEVEL SNAPSHOT");
+    Assert.assertEquals("AST doesn't match",
+      "(TOK_START_TRANSACTION (TOK_TXN_ACCESS_MODE TOK_TXN_READ_WRITE) (TOK_ISOLATION_LEVEL TOK_ISOLATION_SNAPSHOT))", ast.toStringTree());
+    
+  }
+  @Test
+  public void testTxnCommitRollback() throws ParseException {
+    ASTNode ast = parse("COMMIT");
+    Assert.assertEquals("AST doesn't match", "TOK_COMMIT", ast.toStringTree());
+    ast = parse("COMMIT WORK");
+    Assert.assertEquals("AST doesn't match", "TOK_COMMIT", ast.toStringTree());
+    ast = parse("ROLLBACK");
+    Assert.assertEquals("AST doesn't match", "TOK_ROLLBACK", ast.toStringTree());
+    ast = parse("ROLLBACK WORK");
+    Assert.assertEquals("AST doesn't match", "TOK_ROLLBACK", ast.toStringTree());
+  }
+  
+  @Test
+  public void testAutoCommit() throws ParseException {
+    ASTNode ast = parse("SET AUTOCOMMIT TRUE");
+    Assert.assertEquals("AST doesn't match", "(TOK_SET_AUTOCOMMIT TOK_TRUE)", ast.toStringTree());
+    ast = parse("SET AUTOCOMMIT FALSE");
+    Assert.assertEquals("AST doesn't match", "(TOK_SET_AUTOCOMMIT TOK_FALSE)", ast.toStringTree());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/e57c3602/ql/src/test/org/apache/hadoop/hive/ql/session/TestSessionState.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/session/TestSessionState.java b/ql/src/test/org/apache/hadoop/hive/ql/session/TestSessionState.java
index 9e16c0c..70985b3 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/session/TestSessionState.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/session/TestSessionState.java
@@ -165,7 +165,7 @@ public class TestSessionState {
     // set state in current thread
     SessionState.start(ss1);
     SessionState ss2 = SessionState.get();
-    ClassLoader loader2 = ss2.conf.getClassLoader();
+    ClassLoader loader2 = ss2.getConf().getClassLoader();
 
     System.out.println("Loader1:(Set in other thread) " + otherThread.loader);
     System.out.println("Loader2:(Set in SessionState.conf) " + loader2);

http://git-wip-us.apache.org/repos/asf/hive/blob/e57c3602/ql/src/test/results/clientnegative/exchange_partition.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/exchange_partition.q.out b/ql/src/test/results/clientnegative/exchange_partition.q.out
index b81fb99..8622615 100644
--- a/ql/src/test/results/clientnegative/exchange_partition.q.out
+++ b/ql/src/test/results/clientnegative/exchange_partition.q.out
@@ -50,5 +50,5 @@ POSTHOOK: type: SHOWPARTITIONS
 POSTHOOK: Input: default@ex_table2
 part=part1
 PREHOOK: query: ALTER TABLE ex_table1 EXCHANGE PARTITION (part='part1') WITH TABLE ex_table2
-PREHOOK: type: null
+PREHOOK: type: ALTERTABLE_EXCHANGEPARTITION
 FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. MetaException(message:Got exception: java.io.IOException Cannot rename the source path. The destination path already exists.)

http://git-wip-us.apache.org/repos/asf/hive/blob/e57c3602/ql/src/test/results/clientpositive/exchange_partition.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/exchange_partition.q.out b/ql/src/test/results/clientpositive/exchange_partition.q.out
index 4ff1f6c..5b21eaf 100644
--- a/ql/src/test/results/clientpositive/exchange_partition.q.out
+++ b/ql/src/test/results/clientpositive/exchange_partition.q.out
@@ -59,9 +59,9 @@ POSTHOOK: type: SHOWPARTITIONS
 POSTHOOK: Input: ex2@exchange_part_test2
 ds=2013-04-05
 PREHOOK: query: ALTER TABLE ex1.exchange_part_test1 EXCHANGE PARTITION (ds='2013-04-05') WITH TABLE ex2.exchange_part_test2
-PREHOOK: type: null
+PREHOOK: type: ALTERTABLE_EXCHANGEPARTITION
 POSTHOOK: query: ALTER TABLE ex1.exchange_part_test1 EXCHANGE PARTITION (ds='2013-04-05') WITH TABLE ex2.exchange_part_test2
-POSTHOOK: type: null
+POSTHOOK: type: ALTERTABLE_EXCHANGEPARTITION
 PREHOOK: query: SHOW PARTITIONS ex1.exchange_part_test1
 PREHOOK: type: SHOWPARTITIONS
 PREHOOK: Input: ex1@exchange_part_test1

http://git-wip-us.apache.org/repos/asf/hive/blob/e57c3602/ql/src/test/results/clientpositive/exchange_partition2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/exchange_partition2.q.out b/ql/src/test/results/clientpositive/exchange_partition2.q.out
index d47fb05..8c7c583 100644
--- a/ql/src/test/results/clientpositive/exchange_partition2.q.out
+++ b/ql/src/test/results/clientpositive/exchange_partition2.q.out
@@ -47,9 +47,9 @@ POSTHOOK: type: SHOWPARTITIONS
 POSTHOOK: Input: default@exchange_part_test2
 ds=2013-04-05/hr=1
 PREHOOK: query: ALTER TABLE exchange_part_test1 EXCHANGE PARTITION (ds='2013-04-05', hr='1') WITH TABLE exchange_part_test2
-PREHOOK: type: null
+PREHOOK: type: ALTERTABLE_EXCHANGEPARTITION
 POSTHOOK: query: ALTER TABLE exchange_part_test1 EXCHANGE PARTITION (ds='2013-04-05', hr='1') WITH TABLE exchange_part_test2
-POSTHOOK: type: null
+POSTHOOK: type: ALTERTABLE_EXCHANGEPARTITION
 PREHOOK: query: SHOW PARTITIONS exchange_part_test1
 PREHOOK: type: SHOWPARTITIONS
 PREHOOK: Input: default@exchange_part_test1

http://git-wip-us.apache.org/repos/asf/hive/blob/e57c3602/ql/src/test/results/clientpositive/exchange_partition3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/exchange_partition3.q.out b/ql/src/test/results/clientpositive/exchange_partition3.q.out
index 3133ad7..3815861 100644
--- a/ql/src/test/results/clientpositive/exchange_partition3.q.out
+++ b/ql/src/test/results/clientpositive/exchange_partition3.q.out
@@ -64,10 +64,10 @@ ds=2013-04-05/hr=1
 ds=2013-04-05/hr=2
 PREHOOK: query: -- This will exchange both partitions hr=1 and hr=2
 ALTER TABLE exchange_part_test1 EXCHANGE PARTITION (ds='2013-04-05') WITH TABLE exchange_part_test2
-PREHOOK: type: null
+PREHOOK: type: ALTERTABLE_EXCHANGEPARTITION
 POSTHOOK: query: -- This will exchange both partitions hr=1 and hr=2
 ALTER TABLE exchange_part_test1 EXCHANGE PARTITION (ds='2013-04-05') WITH TABLE exchange_part_test2
-POSTHOOK: type: null
+POSTHOOK: type: ALTERTABLE_EXCHANGEPARTITION
 PREHOOK: query: SHOW PARTITIONS exchange_part_test1
 PREHOOK: type: SHOWPARTITIONS
 PREHOOK: Input: default@exchange_part_test1


[38/43] hive git commit: HIVE-11143: Tests udf_from_utc_timestamp.q/udf_to_utc_timestamp.q do not work with updated Java timezone information (Jason Dere, reviewed by Alex Pivovarov)

Posted by xu...@apache.org.
HIVE-11143: Tests udf_from_utc_timestamp.q/udf_to_utc_timestamp.q do not work with updated Java timezone information (Jason Dere, reviewed by Alex Pivovarov)


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

Branch: refs/heads/spark
Commit: f312d17ef19d79a3b67792a67afd8435f90681ef
Parents: 9ae70cb
Author: Jason Dere <jd...@hortonworks.com>
Authored: Wed Jul 29 14:26:16 2015 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Wed Jul 29 14:26:16 2015 -0700

----------------------------------------------------------------------
 .../clientpositive/udf_from_utc_timestamp.q     | 30 ++++-----
 .../clientpositive/udf_to_utc_timestamp.q       | 30 ++++-----
 .../clientpositive/udf_from_utc_timestamp.q.out | 66 ++++++++++----------
 .../clientpositive/udf_to_utc_timestamp.q.out   | 66 ++++++++++----------
 4 files changed, 96 insertions(+), 96 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/f312d17e/ql/src/test/queries/clientpositive/udf_from_utc_timestamp.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/udf_from_utc_timestamp.q b/ql/src/test/queries/clientpositive/udf_from_utc_timestamp.q
index de98507..b113eec 100644
--- a/ql/src/test/queries/clientpositive/udf_from_utc_timestamp.q
+++ b/ql/src/test/queries/clientpositive/udf_from_utc_timestamp.q
@@ -1,24 +1,24 @@
 DESCRIBE FUNCTION from_utc_timestamp;
 DESC FUNCTION EXTENDED from_utc_timestamp;
 
-explain select from_utc_timestamp('2015-02-11 10:30:00', 'PST');
+explain select from_utc_timestamp('2012-02-11 10:30:00', 'PST');
 
 select
-from_utc_timestamp('2015-02-11 04:30:00', 'PST'),
-from_utc_timestamp('2015-02-11 04:30:00', 'Europe/Moscow'),
-from_utc_timestamp('2015-02-11 04:30:00', 'GMT+8'),
-from_utc_timestamp('2015-02-11 04:30:00', 'GMT'),
-from_utc_timestamp('2015-02-11 04:30:00', ''),
-from_utc_timestamp('2015-02-11 04:30:00', '---'),
+from_utc_timestamp('2012-02-11 04:30:00', 'PST'),
+from_utc_timestamp('2012-02-11 04:30:00', 'Europe/Moscow'),
+from_utc_timestamp('2012-02-11 04:30:00', 'GMT+8'),
+from_utc_timestamp('2012-02-11 04:30:00', 'GMT'),
+from_utc_timestamp('2012-02-11 04:30:00', ''),
+from_utc_timestamp('2012-02-11 04:30:00', '---'),
 from_utc_timestamp(cast(null as string), 'PST'),
-from_utc_timestamp('2015-02-11 04:30:00', cast(null as string));
+from_utc_timestamp('2012-02-11 04:30:00', cast(null as string));
 
 select
-from_utc_timestamp(cast('2015-02-11 04:30:00' as timestamp), 'PST'),
-from_utc_timestamp(cast('2015-02-11 04:30:00' as timestamp), 'Europe/Moscow'),
-from_utc_timestamp(cast('2015-02-11 04:30:00' as timestamp), 'GMT+8'),
-from_utc_timestamp(cast('2015-02-11 04:30:00' as timestamp), 'GMT'),
-from_utc_timestamp(cast('2015-02-11 04:30:00' as timestamp), ''),
-from_utc_timestamp(cast('2015-02-11 04:30:00' as timestamp), '---'),
+from_utc_timestamp(cast('2012-02-11 04:30:00' as timestamp), 'PST'),
+from_utc_timestamp(cast('2012-02-11 04:30:00' as timestamp), 'Europe/Moscow'),
+from_utc_timestamp(cast('2012-02-11 04:30:00' as timestamp), 'GMT+8'),
+from_utc_timestamp(cast('2012-02-11 04:30:00' as timestamp), 'GMT'),
+from_utc_timestamp(cast('2012-02-11 04:30:00' as timestamp), ''),
+from_utc_timestamp(cast('2012-02-11 04:30:00' as timestamp), '---'),
 from_utc_timestamp(cast(null as timestamp), 'PST'),
-from_utc_timestamp(cast('2015-02-11 04:30:00' as timestamp), cast(null as string));
\ No newline at end of file
+from_utc_timestamp(cast('2012-02-11 04:30:00' as timestamp), cast(null as string));

http://git-wip-us.apache.org/repos/asf/hive/blob/f312d17e/ql/src/test/queries/clientpositive/udf_to_utc_timestamp.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/udf_to_utc_timestamp.q b/ql/src/test/queries/clientpositive/udf_to_utc_timestamp.q
index fe0b647..cca6d7d 100644
--- a/ql/src/test/queries/clientpositive/udf_to_utc_timestamp.q
+++ b/ql/src/test/queries/clientpositive/udf_to_utc_timestamp.q
@@ -1,24 +1,24 @@
 DESCRIBE FUNCTION to_utc_timestamp;
 DESC FUNCTION EXTENDED to_utc_timestamp;
 
-explain select to_utc_timestamp('2015-02-11 10:30:00', 'PST');
+explain select to_utc_timestamp('2012-02-11 10:30:00', 'PST');
 
 select
-to_utc_timestamp('2015-02-10 20:30:00', 'PST'),
-to_utc_timestamp('2015-02-11 08:30:00', 'Europe/Moscow'),
-to_utc_timestamp('2015-02-11 12:30:00', 'GMT+8'),
-to_utc_timestamp('2015-02-11 04:30:00', 'GMT'),
-to_utc_timestamp('2015-02-11 04:30:00', ''),
-to_utc_timestamp('2015-02-11 04:30:00', '---'),
+to_utc_timestamp('2012-02-10 20:30:00', 'PST'),
+to_utc_timestamp('2012-02-11 08:30:00', 'Europe/Moscow'),
+to_utc_timestamp('2012-02-11 12:30:00', 'GMT+8'),
+to_utc_timestamp('2012-02-11 04:30:00', 'GMT'),
+to_utc_timestamp('2012-02-11 04:30:00', ''),
+to_utc_timestamp('2012-02-11 04:30:00', '---'),
 to_utc_timestamp(cast(null as string), 'PST'),
-to_utc_timestamp('2015-02-11 04:30:00', cast(null as string));
+to_utc_timestamp('2012-02-11 04:30:00', cast(null as string));
 
 select
-to_utc_timestamp(cast('2015-02-10 20:30:00' as timestamp), 'PST'),
-to_utc_timestamp(cast('2015-02-11 08:30:00' as timestamp), 'Europe/Moscow'),
-to_utc_timestamp(cast('2015-02-11 12:30:00' as timestamp), 'GMT+8'),
-to_utc_timestamp(cast('2015-02-11 04:30:00' as timestamp), 'GMT'),
-to_utc_timestamp(cast('2015-02-11 04:30:00' as timestamp), ''),
-to_utc_timestamp(cast('2015-02-11 04:30:00' as timestamp), '---'),
+to_utc_timestamp(cast('2012-02-10 20:30:00' as timestamp), 'PST'),
+to_utc_timestamp(cast('2012-02-11 08:30:00' as timestamp), 'Europe/Moscow'),
+to_utc_timestamp(cast('2012-02-11 12:30:00' as timestamp), 'GMT+8'),
+to_utc_timestamp(cast('2012-02-11 04:30:00' as timestamp), 'GMT'),
+to_utc_timestamp(cast('2012-02-11 04:30:00' as timestamp), ''),
+to_utc_timestamp(cast('2012-02-11 04:30:00' as timestamp), '---'),
 to_utc_timestamp(cast(null as timestamp), 'PST'),
-to_utc_timestamp(cast('2015-02-11 04:30:00' as timestamp), cast(null as string));
\ No newline at end of file
+to_utc_timestamp(cast('2012-02-11 04:30:00' as timestamp), cast(null as string));

http://git-wip-us.apache.org/repos/asf/hive/blob/f312d17e/ql/src/test/results/clientpositive/udf_from_utc_timestamp.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/udf_from_utc_timestamp.q.out b/ql/src/test/results/clientpositive/udf_from_utc_timestamp.q.out
index 577d41c..5b4bc35 100644
--- a/ql/src/test/results/clientpositive/udf_from_utc_timestamp.q.out
+++ b/ql/src/test/results/clientpositive/udf_from_utc_timestamp.q.out
@@ -8,9 +8,9 @@ PREHOOK: type: DESCFUNCTION
 POSTHOOK: query: DESC FUNCTION EXTENDED from_utc_timestamp
 POSTHOOK: type: DESCFUNCTION
 from_utc_timestamp(timestamp, string timezone) - Assumes given timestamp is UTC and converts to given timezone (as of Hive 0.8.0)
-PREHOOK: query: explain select from_utc_timestamp('2015-02-11 10:30:00', 'PST')
+PREHOOK: query: explain select from_utc_timestamp('2012-02-11 10:30:00', 'PST')
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select from_utc_timestamp('2015-02-11 10:30:00', 'PST')
+POSTHOOK: query: explain select from_utc_timestamp('2012-02-11 10:30:00', 'PST')
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
@@ -25,58 +25,58 @@ STAGE PLANS:
           Row Limit Per Split: 1
           Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
-            expressions: 2015-02-11 02:30:00.0 (type: timestamp)
+            expressions: 2012-02-11 02:30:00.0 (type: timestamp)
             outputColumnNames: _col0
             Statistics: Num rows: 1 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             ListSink
 
 PREHOOK: query: select
-from_utc_timestamp('2015-02-11 04:30:00', 'PST'),
-from_utc_timestamp('2015-02-11 04:30:00', 'Europe/Moscow'),
-from_utc_timestamp('2015-02-11 04:30:00', 'GMT+8'),
-from_utc_timestamp('2015-02-11 04:30:00', 'GMT'),
-from_utc_timestamp('2015-02-11 04:30:00', ''),
-from_utc_timestamp('2015-02-11 04:30:00', '---'),
+from_utc_timestamp('2012-02-11 04:30:00', 'PST'),
+from_utc_timestamp('2012-02-11 04:30:00', 'Europe/Moscow'),
+from_utc_timestamp('2012-02-11 04:30:00', 'GMT+8'),
+from_utc_timestamp('2012-02-11 04:30:00', 'GMT'),
+from_utc_timestamp('2012-02-11 04:30:00', ''),
+from_utc_timestamp('2012-02-11 04:30:00', '---'),
 from_utc_timestamp(cast(null as string), 'PST'),
-from_utc_timestamp('2015-02-11 04:30:00', cast(null as string))
+from_utc_timestamp('2012-02-11 04:30:00', cast(null as string))
 PREHOOK: type: QUERY
 PREHOOK: Input: _dummy_database@_dummy_table
 #### A masked pattern was here ####
 POSTHOOK: query: select
-from_utc_timestamp('2015-02-11 04:30:00', 'PST'),
-from_utc_timestamp('2015-02-11 04:30:00', 'Europe/Moscow'),
-from_utc_timestamp('2015-02-11 04:30:00', 'GMT+8'),
-from_utc_timestamp('2015-02-11 04:30:00', 'GMT'),
-from_utc_timestamp('2015-02-11 04:30:00', ''),
-from_utc_timestamp('2015-02-11 04:30:00', '---'),
+from_utc_timestamp('2012-02-11 04:30:00', 'PST'),
+from_utc_timestamp('2012-02-11 04:30:00', 'Europe/Moscow'),
+from_utc_timestamp('2012-02-11 04:30:00', 'GMT+8'),
+from_utc_timestamp('2012-02-11 04:30:00', 'GMT'),
+from_utc_timestamp('2012-02-11 04:30:00', ''),
+from_utc_timestamp('2012-02-11 04:30:00', '---'),
 from_utc_timestamp(cast(null as string), 'PST'),
-from_utc_timestamp('2015-02-11 04:30:00', cast(null as string))
+from_utc_timestamp('2012-02-11 04:30:00', cast(null as string))
 POSTHOOK: type: QUERY
 POSTHOOK: Input: _dummy_database@_dummy_table
 #### A masked pattern was here ####
-2015-02-10 20:30:00	2015-02-11 08:30:00	2015-02-11 12:30:00	2015-02-11 04:30:00	2015-02-11 04:30:00	2015-02-11 04:30:00	NULL	NULL
+2012-02-10 20:30:00	2012-02-11 08:30:00	2012-02-11 12:30:00	2012-02-11 04:30:00	2012-02-11 04:30:00	2012-02-11 04:30:00	NULL	NULL
 PREHOOK: query: select
-from_utc_timestamp(cast('2015-02-11 04:30:00' as timestamp), 'PST'),
-from_utc_timestamp(cast('2015-02-11 04:30:00' as timestamp), 'Europe/Moscow'),
-from_utc_timestamp(cast('2015-02-11 04:30:00' as timestamp), 'GMT+8'),
-from_utc_timestamp(cast('2015-02-11 04:30:00' as timestamp), 'GMT'),
-from_utc_timestamp(cast('2015-02-11 04:30:00' as timestamp), ''),
-from_utc_timestamp(cast('2015-02-11 04:30:00' as timestamp), '---'),
+from_utc_timestamp(cast('2012-02-11 04:30:00' as timestamp), 'PST'),
+from_utc_timestamp(cast('2012-02-11 04:30:00' as timestamp), 'Europe/Moscow'),
+from_utc_timestamp(cast('2012-02-11 04:30:00' as timestamp), 'GMT+8'),
+from_utc_timestamp(cast('2012-02-11 04:30:00' as timestamp), 'GMT'),
+from_utc_timestamp(cast('2012-02-11 04:30:00' as timestamp), ''),
+from_utc_timestamp(cast('2012-02-11 04:30:00' as timestamp), '---'),
 from_utc_timestamp(cast(null as timestamp), 'PST'),
-from_utc_timestamp(cast('2015-02-11 04:30:00' as timestamp), cast(null as string))
+from_utc_timestamp(cast('2012-02-11 04:30:00' as timestamp), cast(null as string))
 PREHOOK: type: QUERY
 PREHOOK: Input: _dummy_database@_dummy_table
 #### A masked pattern was here ####
 POSTHOOK: query: select
-from_utc_timestamp(cast('2015-02-11 04:30:00' as timestamp), 'PST'),
-from_utc_timestamp(cast('2015-02-11 04:30:00' as timestamp), 'Europe/Moscow'),
-from_utc_timestamp(cast('2015-02-11 04:30:00' as timestamp), 'GMT+8'),
-from_utc_timestamp(cast('2015-02-11 04:30:00' as timestamp), 'GMT'),
-from_utc_timestamp(cast('2015-02-11 04:30:00' as timestamp), ''),
-from_utc_timestamp(cast('2015-02-11 04:30:00' as timestamp), '---'),
+from_utc_timestamp(cast('2012-02-11 04:30:00' as timestamp), 'PST'),
+from_utc_timestamp(cast('2012-02-11 04:30:00' as timestamp), 'Europe/Moscow'),
+from_utc_timestamp(cast('2012-02-11 04:30:00' as timestamp), 'GMT+8'),
+from_utc_timestamp(cast('2012-02-11 04:30:00' as timestamp), 'GMT'),
+from_utc_timestamp(cast('2012-02-11 04:30:00' as timestamp), ''),
+from_utc_timestamp(cast('2012-02-11 04:30:00' as timestamp), '---'),
 from_utc_timestamp(cast(null as timestamp), 'PST'),
-from_utc_timestamp(cast('2015-02-11 04:30:00' as timestamp), cast(null as string))
+from_utc_timestamp(cast('2012-02-11 04:30:00' as timestamp), cast(null as string))
 POSTHOOK: type: QUERY
 POSTHOOK: Input: _dummy_database@_dummy_table
 #### A masked pattern was here ####
-2015-02-10 20:30:00	2015-02-11 08:30:00	2015-02-11 12:30:00	2015-02-11 04:30:00	2015-02-11 04:30:00	2015-02-11 04:30:00	NULL	NULL
+2012-02-10 20:30:00	2012-02-11 08:30:00	2012-02-11 12:30:00	2012-02-11 04:30:00	2012-02-11 04:30:00	2012-02-11 04:30:00	NULL	NULL

http://git-wip-us.apache.org/repos/asf/hive/blob/f312d17e/ql/src/test/results/clientpositive/udf_to_utc_timestamp.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/udf_to_utc_timestamp.q.out b/ql/src/test/results/clientpositive/udf_to_utc_timestamp.q.out
index 81ecd44..1730f8c 100644
--- a/ql/src/test/results/clientpositive/udf_to_utc_timestamp.q.out
+++ b/ql/src/test/results/clientpositive/udf_to_utc_timestamp.q.out
@@ -8,9 +8,9 @@ PREHOOK: type: DESCFUNCTION
 POSTHOOK: query: DESC FUNCTION EXTENDED to_utc_timestamp
 POSTHOOK: type: DESCFUNCTION
 to_utc_timestamp(timestamp, string timezone) - Assumes given timestamp is in given timezone and converts to UTC (as of Hive 0.8.0)
-PREHOOK: query: explain select to_utc_timestamp('2015-02-11 10:30:00', 'PST')
+PREHOOK: query: explain select to_utc_timestamp('2012-02-11 10:30:00', 'PST')
 PREHOOK: type: QUERY
-POSTHOOK: query: explain select to_utc_timestamp('2015-02-11 10:30:00', 'PST')
+POSTHOOK: query: explain select to_utc_timestamp('2012-02-11 10:30:00', 'PST')
 POSTHOOK: type: QUERY
 STAGE DEPENDENCIES:
   Stage-0 is a root stage
@@ -25,58 +25,58 @@ STAGE PLANS:
           Row Limit Per Split: 1
           Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
-            expressions: 2015-02-11 18:30:00.0 (type: timestamp)
+            expressions: 2012-02-11 18:30:00.0 (type: timestamp)
             outputColumnNames: _col0
             Statistics: Num rows: 1 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
             ListSink
 
 PREHOOK: query: select
-to_utc_timestamp('2015-02-10 20:30:00', 'PST'),
-to_utc_timestamp('2015-02-11 08:30:00', 'Europe/Moscow'),
-to_utc_timestamp('2015-02-11 12:30:00', 'GMT+8'),
-to_utc_timestamp('2015-02-11 04:30:00', 'GMT'),
-to_utc_timestamp('2015-02-11 04:30:00', ''),
-to_utc_timestamp('2015-02-11 04:30:00', '---'),
+to_utc_timestamp('2012-02-10 20:30:00', 'PST'),
+to_utc_timestamp('2012-02-11 08:30:00', 'Europe/Moscow'),
+to_utc_timestamp('2012-02-11 12:30:00', 'GMT+8'),
+to_utc_timestamp('2012-02-11 04:30:00', 'GMT'),
+to_utc_timestamp('2012-02-11 04:30:00', ''),
+to_utc_timestamp('2012-02-11 04:30:00', '---'),
 to_utc_timestamp(cast(null as string), 'PST'),
-to_utc_timestamp('2015-02-11 04:30:00', cast(null as string))
+to_utc_timestamp('2012-02-11 04:30:00', cast(null as string))
 PREHOOK: type: QUERY
 PREHOOK: Input: _dummy_database@_dummy_table
 #### A masked pattern was here ####
 POSTHOOK: query: select
-to_utc_timestamp('2015-02-10 20:30:00', 'PST'),
-to_utc_timestamp('2015-02-11 08:30:00', 'Europe/Moscow'),
-to_utc_timestamp('2015-02-11 12:30:00', 'GMT+8'),
-to_utc_timestamp('2015-02-11 04:30:00', 'GMT'),
-to_utc_timestamp('2015-02-11 04:30:00', ''),
-to_utc_timestamp('2015-02-11 04:30:00', '---'),
+to_utc_timestamp('2012-02-10 20:30:00', 'PST'),
+to_utc_timestamp('2012-02-11 08:30:00', 'Europe/Moscow'),
+to_utc_timestamp('2012-02-11 12:30:00', 'GMT+8'),
+to_utc_timestamp('2012-02-11 04:30:00', 'GMT'),
+to_utc_timestamp('2012-02-11 04:30:00', ''),
+to_utc_timestamp('2012-02-11 04:30:00', '---'),
 to_utc_timestamp(cast(null as string), 'PST'),
-to_utc_timestamp('2015-02-11 04:30:00', cast(null as string))
+to_utc_timestamp('2012-02-11 04:30:00', cast(null as string))
 POSTHOOK: type: QUERY
 POSTHOOK: Input: _dummy_database@_dummy_table
 #### A masked pattern was here ####
-2015-02-11 04:30:00	2015-02-11 04:30:00	2015-02-11 04:30:00	2015-02-11 04:30:00	2015-02-11 04:30:00	2015-02-11 04:30:00	NULL	NULL
+2012-02-11 04:30:00	2012-02-11 04:30:00	2012-02-11 04:30:00	2012-02-11 04:30:00	2012-02-11 04:30:00	2012-02-11 04:30:00	NULL	NULL
 PREHOOK: query: select
-to_utc_timestamp(cast('2015-02-10 20:30:00' as timestamp), 'PST'),
-to_utc_timestamp(cast('2015-02-11 08:30:00' as timestamp), 'Europe/Moscow'),
-to_utc_timestamp(cast('2015-02-11 12:30:00' as timestamp), 'GMT+8'),
-to_utc_timestamp(cast('2015-02-11 04:30:00' as timestamp), 'GMT'),
-to_utc_timestamp(cast('2015-02-11 04:30:00' as timestamp), ''),
-to_utc_timestamp(cast('2015-02-11 04:30:00' as timestamp), '---'),
+to_utc_timestamp(cast('2012-02-10 20:30:00' as timestamp), 'PST'),
+to_utc_timestamp(cast('2012-02-11 08:30:00' as timestamp), 'Europe/Moscow'),
+to_utc_timestamp(cast('2012-02-11 12:30:00' as timestamp), 'GMT+8'),
+to_utc_timestamp(cast('2012-02-11 04:30:00' as timestamp), 'GMT'),
+to_utc_timestamp(cast('2012-02-11 04:30:00' as timestamp), ''),
+to_utc_timestamp(cast('2012-02-11 04:30:00' as timestamp), '---'),
 to_utc_timestamp(cast(null as timestamp), 'PST'),
-to_utc_timestamp(cast('2015-02-11 04:30:00' as timestamp), cast(null as string))
+to_utc_timestamp(cast('2012-02-11 04:30:00' as timestamp), cast(null as string))
 PREHOOK: type: QUERY
 PREHOOK: Input: _dummy_database@_dummy_table
 #### A masked pattern was here ####
 POSTHOOK: query: select
-to_utc_timestamp(cast('2015-02-10 20:30:00' as timestamp), 'PST'),
-to_utc_timestamp(cast('2015-02-11 08:30:00' as timestamp), 'Europe/Moscow'),
-to_utc_timestamp(cast('2015-02-11 12:30:00' as timestamp), 'GMT+8'),
-to_utc_timestamp(cast('2015-02-11 04:30:00' as timestamp), 'GMT'),
-to_utc_timestamp(cast('2015-02-11 04:30:00' as timestamp), ''),
-to_utc_timestamp(cast('2015-02-11 04:30:00' as timestamp), '---'),
+to_utc_timestamp(cast('2012-02-10 20:30:00' as timestamp), 'PST'),
+to_utc_timestamp(cast('2012-02-11 08:30:00' as timestamp), 'Europe/Moscow'),
+to_utc_timestamp(cast('2012-02-11 12:30:00' as timestamp), 'GMT+8'),
+to_utc_timestamp(cast('2012-02-11 04:30:00' as timestamp), 'GMT'),
+to_utc_timestamp(cast('2012-02-11 04:30:00' as timestamp), ''),
+to_utc_timestamp(cast('2012-02-11 04:30:00' as timestamp), '---'),
 to_utc_timestamp(cast(null as timestamp), 'PST'),
-to_utc_timestamp(cast('2015-02-11 04:30:00' as timestamp), cast(null as string))
+to_utc_timestamp(cast('2012-02-11 04:30:00' as timestamp), cast(null as string))
 POSTHOOK: type: QUERY
 POSTHOOK: Input: _dummy_database@_dummy_table
 #### A masked pattern was here ####
-2015-02-11 04:30:00	2015-02-11 04:30:00	2015-02-11 04:30:00	2015-02-11 04:30:00	2015-02-11 04:30:00	2015-02-11 04:30:00	NULL	NULL
+2012-02-11 04:30:00	2012-02-11 04:30:00	2012-02-11 04:30:00	2012-02-11 04:30:00	2012-02-11 04:30:00	2012-02-11 04:30:00	NULL	NULL


[30/43] hive git commit: HIVE-10799. Refactor the SearchArgumentFactory to remove the AST-specific factory. (omalley reviewed by prasanth_j)

Posted by xu...@apache.org.
HIVE-10799. Refactor the SearchArgumentFactory to remove the AST-specific factory. (omalley reviewed by prasanth_j)


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

Branch: refs/heads/spark
Commit: c178a6e9d12055e5bde634123ca58f243ae39477
Parents: d2ee354
Author: Owen O'Malley <om...@apache.org>
Authored: Tue Jul 28 12:47:59 2015 -0700
Committer: Owen O'Malley <om...@apache.org>
Committed: Tue Jul 28 12:47:59 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hive/common/type/HiveDecimal.java    |    2 +-
 .../hadoop/hive/ql/io/orc/OrcInputFormat.java   |    4 +-
 .../hadoop/hive/ql/io/orc/RecordReaderImpl.java |   30 +-
 .../read/ParquetRecordReaderWrapper.java        |    7 +-
 .../hive/ql/io/sarg/ConvertAstToSearchArg.java  |  439 +++
 .../hive/ql/io/sarg/SearchArgumentFactory.java  |   25 +-
 .../hive/ql/io/sarg/SearchArgumentImpl.java     | 1000 ++----
 .../hive/ql/io/orc/TestInputOutputFormat.java   |    9 +-
 .../hadoop/hive/ql/io/orc/TestOrcFile.java      |   11 +-
 .../hive/ql/io/orc/TestRecordReaderImpl.java    |   63 +-
 .../parquet/TestParquetRecordReaderWrapper.java |  155 +
 .../ql/io/sarg/TestConvertAstToSearchArg.java   | 2863 +++++++++++++++++
 .../hive/ql/io/sarg/TestSearchArgumentImpl.java | 2888 +-----------------
 .../hadoop/hive/ql/io/sarg/PredicateLeaf.java   |   33 +-
 .../hadoop/hive/ql/io/sarg/SearchArgument.java  |   34 +-
 .../hive/serde2/io/HiveDecimalWritable.java     |    8 +
 16 files changed, 4080 insertions(+), 3491 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/c178a6e9/common/src/java/org/apache/hadoop/hive/common/type/HiveDecimal.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/type/HiveDecimal.java b/common/src/java/org/apache/hadoop/hive/common/type/HiveDecimal.java
index a8215f2..f14fc2d 100644
--- a/common/src/java/org/apache/hadoop/hive/common/type/HiveDecimal.java
+++ b/common/src/java/org/apache/hadoop/hive/common/type/HiveDecimal.java
@@ -75,7 +75,7 @@ public class HiveDecimal implements Comparable<HiveDecimal> {
   public static HiveDecimal create(String dec) {
     BigDecimal bd;
     try {
-      bd = new BigDecimal(dec);
+      bd = new BigDecimal(dec.trim());
     } catch (NumberFormatException ex) {
       return null;
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/c178a6e9/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
index 3a9e64e..4e6dd7a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcInputFormat.java
@@ -54,10 +54,10 @@ import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
 import org.apache.hadoop.hive.ql.io.InputFormatChecker;
 import org.apache.hadoop.hive.ql.io.RecordIdentifier;
 import org.apache.hadoop.hive.ql.io.StatsProvidingRecordReader;
+import org.apache.hadoop.hive.ql.io.sarg.ConvertAstToSearchArg;
 import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument.TruthValue;
-import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory;
 import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
 import org.apache.hadoop.hive.serde2.SerDeStats;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
@@ -305,7 +305,7 @@ public class OrcInputFormat  implements InputFormat<NullWritable, OrcStruct>,
       options.searchArgument(null, null);
       return;
     }
-    SearchArgument sarg = SearchArgumentFactory.createFromConf(conf);
+    SearchArgument sarg = ConvertAstToSearchArg.createFromConf(conf);
     if (sarg == null) {
       LOG.debug("No ORC pushdown predicate");
       options.searchArgument(null, null);

http://git-wip-us.apache.org/repos/asf/hive/blob/c178a6e9/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
index 4f79e37..f85420d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
@@ -47,8 +47,8 @@ import org.apache.hadoop.hive.ql.io.orc.TreeReaderFactory.TreeReader;
 import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument.TruthValue;
-import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.serde2.io.TimestampWritable;
 import org.apache.hadoop.hive.shims.HadoopShims.ZeroCopyReaderShim;
 import org.apache.hadoop.io.Text;
@@ -523,7 +523,8 @@ class RecordReaderImpl implements RecordReader {
         result = TruthValue.YES_NO_NULL;
       }
     } else if (predObj instanceof String || predObj instanceof Text ||
-        predObj instanceof HiveDecimal || predObj instanceof BigDecimal) {
+        predObj instanceof HiveDecimalWritable ||
+        predObj instanceof BigDecimal) {
       if (bf.testString(predObj.toString())) {
         result = TruthValue.YES_NO_NULL;
       }
@@ -560,11 +561,7 @@ class RecordReaderImpl implements RecordReader {
   }
 
   private static Object getBaseObjectForComparison(PredicateLeaf.Type type, Object obj) {
-    if (obj != null) {
-      if (obj instanceof ExprNodeConstantDesc) {
-        obj = ((ExprNodeConstantDesc) obj).getValue();
-      }
-    } else {
+    if (obj == null) {
       return null;
     }
     switch (type) {
@@ -588,20 +585,23 @@ class RecordReaderImpl implements RecordReader {
         break;
       case DECIMAL:
         if (obj instanceof Boolean) {
-          return ((Boolean) obj).booleanValue() ? HiveDecimal.ONE : HiveDecimal.ZERO;
+          return new HiveDecimalWritable(((Boolean) obj).booleanValue() ?
+              HiveDecimal.ONE : HiveDecimal.ZERO);
         } else if (obj instanceof Integer) {
-          return HiveDecimal.create(((Integer) obj).intValue());
+          return new HiveDecimalWritable(((Integer) obj).intValue());
         } else if (obj instanceof Long) {
-          return HiveDecimal.create(((Long) obj));
+          return new HiveDecimalWritable(((Long) obj));
         } else if (obj instanceof Float || obj instanceof Double ||
             obj instanceof String) {
-          return HiveDecimal.create(obj.toString());
+          return new HiveDecimalWritable(obj.toString());
         } else if (obj instanceof BigDecimal) {
-          return HiveDecimal.create((BigDecimal) obj);
+          return new HiveDecimalWritable(HiveDecimal.create((BigDecimal) obj));
         } else if (obj instanceof HiveDecimal) {
+          return new HiveDecimalWritable((HiveDecimal) obj);
+        } else if (obj instanceof HiveDecimalWritable) {
           return obj;
         } else if (obj instanceof Timestamp) {
-          return HiveDecimal.create(
+          return new HiveDecimalWritable(
               new Double(new TimestampWritable((Timestamp) obj).getDouble()).toString());
         }
         break;
@@ -641,12 +641,16 @@ class RecordReaderImpl implements RecordReader {
       case TIMESTAMP:
         if (obj instanceof Timestamp) {
           return obj;
+        } else if (obj instanceof Integer) {
+          return TimestampWritable.longToTimestamp(((Number) obj).longValue(), false);
         } else if (obj instanceof Float) {
           return TimestampWritable.doubleToTimestamp(((Float) obj).doubleValue());
         } else if (obj instanceof Double) {
           return TimestampWritable.doubleToTimestamp(((Double) obj).doubleValue());
         } else if (obj instanceof HiveDecimal) {
           return TimestampWritable.decimalToTimestamp((HiveDecimal) obj);
+        } else if (obj instanceof HiveDecimalWritable) {
+          return TimestampWritable.decimalToTimestamp(((HiveDecimalWritable) obj).getHiveDecimal());
         } else if (obj instanceof Date) {
           return new Timestamp(((Date) obj).getTime());
         }

http://git-wip-us.apache.org/repos/asf/hive/blob/c178a6e9/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/ParquetRecordReaderWrapper.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/ParquetRecordReaderWrapper.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/ParquetRecordReaderWrapper.java
index a64ec06..49e52da 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/ParquetRecordReaderWrapper.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/ParquetRecordReaderWrapper.java
@@ -27,10 +27,10 @@ import org.apache.hadoop.hive.ql.io.IOConstants;
 import org.apache.hadoop.hive.ql.io.parquet.FilterPredicateLeafBuilder;
 import org.apache.hadoop.hive.ql.io.parquet.LeafFilterFactory;
 import org.apache.hadoop.hive.ql.io.parquet.ProjectionPusher;
+import org.apache.hadoop.hive.ql.io.sarg.ConvertAstToSearchArg;
 import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
 import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
-import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory;
 import org.apache.hadoop.hive.ql.plan.TableScanDesc;
 import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
 import org.apache.hadoop.io.ArrayWritable;
@@ -149,7 +149,7 @@ public class ParquetRecordReaderWrapper  implements RecordReader<NullWritable, A
     }
 
     SearchArgument sarg =
-        SearchArgumentFactory.create(Utilities.deserializeExpression
+        ConvertAstToSearchArg.create(Utilities.deserializeExpression
             (serializedPushdown));
     FilterPredicate p = toFilterPredicate(sarg);
     if (p != null) {
@@ -316,8 +316,7 @@ public class ParquetRecordReaderWrapper  implements RecordReader<NullWritable, A
    * @return translate the sarg into a filter predicate
    */
   public static FilterPredicate toFilterPredicate(SearchArgument sarg) {
-    return translate(sarg.getExpression(),
-        sarg.getLeaves());
+    return translate(sarg.getExpression(), sarg.getLeaves());
   }
 
   private static boolean isMultiLiteralsOperator(PredicateLeaf.Operator op) {

http://git-wip-us.apache.org/repos/asf/hive/blob/c178a6e9/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/ConvertAstToSearchArg.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/ConvertAstToSearchArg.java b/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/ConvertAstToSearchArg.java
new file mode 100644
index 0000000..eb8c03f
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/ConvertAstToSearchArg.java
@@ -0,0 +1,439 @@
+/**
+ * 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.io.sarg;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.io.Input;
+import org.apache.commons.codec.binary.Base64;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.common.type.HiveChar;
+import org.apache.hadoop.hive.common.type.HiveVarchar;
+import org.apache.hadoop.hive.ql.exec.Utilities;
+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.ExprNodeGenericFuncDesc;
+import org.apache.hadoop.hive.ql.plan.TableScanDesc;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBetween;
+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.GenericUDFOPEqualNS;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrGreaterThan;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrLessThan;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPGreaterThan;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPLessThan;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNot;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNotEqual;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNotNull;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNull;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+
+public class ConvertAstToSearchArg {
+  private static final Log LOG = LogFactory.getLog(ConvertAstToSearchArg.class);
+  private final SearchArgument.Builder builder =
+      SearchArgumentFactory.newBuilder();
+
+  /**
+   * Builds the expression and leaf list from the original predicate.
+   * @param expression the expression to translate.
+   */
+  ConvertAstToSearchArg(ExprNodeGenericFuncDesc expression) {
+    parse(expression);
+  }
+
+  /**
+   * Build the search argument from the expression.
+   * @return the search argument
+   */
+  public SearchArgument buildSearchArgument() {
+    return builder.build();
+  }
+
+  /**
+   * Get the type of the given expression node.
+   * @param expr the expression to get the type of
+   * @return int, string, or float or null if we don't know the type
+   */
+  private static PredicateLeaf.Type getType(ExprNodeDesc expr) {
+    TypeInfo type = expr.getTypeInfo();
+    if (type.getCategory() == ObjectInspector.Category.PRIMITIVE) {
+      switch (((PrimitiveTypeInfo) type).getPrimitiveCategory()) {
+        case BYTE:
+        case SHORT:
+        case INT:
+          return PredicateLeaf.Type.INTEGER;
+        case LONG:
+          return PredicateLeaf.Type.LONG;
+        case CHAR:
+        case VARCHAR:
+        case STRING:
+          return PredicateLeaf.Type.STRING;
+        case FLOAT:
+        case DOUBLE:
+          return PredicateLeaf.Type.FLOAT;
+        case DATE:
+          return PredicateLeaf.Type.DATE;
+        case TIMESTAMP:
+          return PredicateLeaf.Type.TIMESTAMP;
+        case DECIMAL:
+          return PredicateLeaf.Type.DECIMAL;
+        case BOOLEAN:
+          return PredicateLeaf.Type.BOOLEAN;
+        default:
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Get the column name referenced in the expression. It must be at the top
+   * level of this expression and there must be exactly one column.
+   * @param expr the expression to look in
+   * @param variable the slot the variable is expected in
+   * @return the column name or null if there isn't exactly one column
+   */
+  private static String getColumnName(ExprNodeGenericFuncDesc expr,
+                                      int variable) {
+    List<ExprNodeDesc> children = expr.getChildren();
+    if (variable < 0 || variable >= children.size()) {
+      return null;
+    }
+    ExprNodeDesc child = children.get(variable);
+    if (child instanceof ExprNodeColumnDesc) {
+      return ((ExprNodeColumnDesc) child).getColumn();
+    }
+    return null;
+  }
+
+  private static Object boxLiteral(ExprNodeConstantDesc constantDesc,
+                                   PredicateLeaf.Type type) {
+    Object lit = constantDesc.getValue();
+    if (lit == null) {
+      return null;
+    }
+    switch (type) {
+      case INTEGER:
+        return ((Number) lit).intValue();
+      case LONG:
+        return ((Number) lit).longValue();
+      case STRING:
+        if (lit instanceof HiveChar) {
+          lit = ((HiveChar) lit).getPaddedValue();
+        } else if (lit instanceof String) {
+          return lit;
+        } else {
+          return lit.toString();
+        }
+      case FLOAT:
+        if (lit instanceof Float) {
+          // converting a float directly to a double causes annoying conversion
+          // problems
+          return Double.parseDouble(lit.toString());
+        } else {
+          return ((Number) lit).doubleValue();
+        }
+      case TIMESTAMP:
+        return Timestamp.valueOf(lit.toString());
+      case DATE:
+        return Date.valueOf(lit.toString());
+      case DECIMAL:
+        LOG.warn("boxing " + lit);
+        return new HiveDecimalWritable(lit.toString());
+      case BOOLEAN:
+        return lit;
+      default:
+        throw new IllegalArgumentException("Unknown literal " + type);
+    }
+  }
+
+  /**
+   * Find the child that is the literal.
+   * @param expr the parent node to check
+   * @param type the type of the expression
+   * @return the literal boxed if found or null
+   */
+  private static Object findLiteral(ExprNodeGenericFuncDesc expr,
+                                    PredicateLeaf.Type type) {
+    List<ExprNodeDesc> children = expr.getChildren();
+    if (children.size() != 2) {
+      return null;
+    }
+    Object result = null;
+    for(ExprNodeDesc child: children) {
+      if (child instanceof ExprNodeConstantDesc) {
+        if (result != null) {
+          return null;
+        }
+        result = boxLiteral((ExprNodeConstantDesc) child, type);
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Return the boxed literal at the given position
+   * @param expr the parent node
+   * @param type the type of the expression
+   * @param position the child position to check
+   * @return the boxed literal if found otherwise null
+   */
+  private static Object getLiteral(ExprNodeGenericFuncDesc expr,
+                                   PredicateLeaf.Type type,
+                                   int position) {
+    List<ExprNodeDesc> children = expr.getChildren();
+    Object child = children.get(position);
+    if (child instanceof ExprNodeConstantDesc) {
+      return boxLiteral((ExprNodeConstantDesc) child, type);
+    }
+    return null;
+  }
+
+  private static Object[] getLiteralList(ExprNodeGenericFuncDesc expr,
+                                         PredicateLeaf.Type type,
+                                         int start) {
+    List<ExprNodeDesc> children = expr.getChildren();
+    Object[] result = new Object[children.size() - start];
+
+    // ignore the first child, since it is the variable
+    int posn = 0;
+    for(ExprNodeDesc child: children.subList(start, children.size())) {
+      if (child instanceof ExprNodeConstantDesc) {
+        result[posn++] = boxLiteral((ExprNodeConstantDesc) child, type);
+      } else {
+        // if we get some non-literals, we need to punt
+        return null;
+      }
+    }
+    return result;
+  }
+
+  private void createLeaf(PredicateLeaf.Operator operator,
+                          ExprNodeGenericFuncDesc expression,
+                          int variable) {
+    String columnName = getColumnName(expression, variable);
+    if (columnName == null) {
+      builder.literal(SearchArgument.TruthValue.YES_NO_NULL);
+      return;
+    }
+    PredicateLeaf.Type type = getType(expression.getChildren().get(variable));
+    if (type == null) {
+      builder.literal(SearchArgument.TruthValue.YES_NO_NULL);
+      return;
+    }
+
+    // if the variable was on the right, we need to swap things around
+    boolean needSwap = false;
+    if (variable != 0) {
+      if (operator == PredicateLeaf.Operator.LESS_THAN) {
+        needSwap = true;
+        operator = PredicateLeaf.Operator.LESS_THAN_EQUALS;
+      } else if (operator == PredicateLeaf.Operator.LESS_THAN_EQUALS) {
+        needSwap = true;
+        operator = PredicateLeaf.Operator.LESS_THAN;
+      }
+    }
+    if (needSwap) {
+      builder.startNot();
+    }
+
+    switch (operator) {
+      case IS_NULL:
+        builder.isNull(columnName, type);
+        break;
+      case EQUALS:
+        builder.equals(columnName, type, findLiteral(expression, type));
+        break;
+      case NULL_SAFE_EQUALS:
+        builder.nullSafeEquals(columnName, type, findLiteral(expression, type));
+        break;
+      case LESS_THAN:
+        builder.lessThan(columnName, type, findLiteral(expression, type));
+        break;
+      case LESS_THAN_EQUALS:
+        builder.lessThanEquals(columnName, type, findLiteral(expression, type));
+        break;
+      case IN:
+        builder.in(columnName, type,
+            getLiteralList(expression, type, variable + 1));
+        break;
+      case BETWEEN:
+        builder.between(columnName, type,
+            getLiteral(expression, type, variable + 1),
+            getLiteral(expression, type, variable + 2));
+        break;
+    }
+
+    if (needSwap) {
+      builder.end();
+    }
+  }
+
+  /**
+   * Find the variable in the expression.
+   * @param expr the expression to look in
+   * @return the index of the variable or -1 if there is not exactly one
+   *   variable.
+   */
+  private int findVariable(ExprNodeDesc expr) {
+    int result = -1;
+    List<ExprNodeDesc> children = expr.getChildren();
+    for(int i = 0; i < children.size(); ++i) {
+      ExprNodeDesc child = children.get(i);
+      if (child instanceof ExprNodeColumnDesc) {
+        // if we already found a variable, this isn't a sarg
+        if (result != -1) {
+          return -1;
+        } else {
+          result = i;
+        }
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Create a leaf expression when we aren't sure where the variable is
+   * located.
+   * @param operator the operator type that was found
+   * @param expression the expression to check
+   */
+  private void createLeaf(PredicateLeaf.Operator operator,
+                          ExprNodeGenericFuncDesc expression) {
+    createLeaf(operator, expression, findVariable(expression));
+  }
+
+  private void addChildren(ExprNodeGenericFuncDesc node) {
+    for(ExprNodeDesc child: node.getChildren()) {
+      parse(child);
+    }
+  }
+
+  /**
+   * Do the recursive parse of the Hive ExprNodeDesc into our ExpressionTree.
+   * @param expression the Hive ExprNodeDesc
+   */
+  private void parse(ExprNodeDesc expression) {
+    // Most of the stuff we can handle are generic function descriptions, so
+    // handle the special cases.
+    if (expression.getClass() != ExprNodeGenericFuncDesc.class) {
+
+      // if it is a reference to a boolean column, covert it to a truth test.
+      if (expression instanceof ExprNodeColumnDesc) {
+        ExprNodeColumnDesc columnDesc = (ExprNodeColumnDesc) expression;
+        if (columnDesc.getTypeString().equals("boolean")) {
+          builder.equals(columnDesc.getColumn(), PredicateLeaf.Type.BOOLEAN,
+              true);
+          return;
+        }
+      }
+
+      // otherwise, we don't know what to do so make it a maybe
+      builder.literal(SearchArgument.TruthValue.YES_NO_NULL);
+      return;
+    }
+
+    // get the kind of expression
+    ExprNodeGenericFuncDesc expr = (ExprNodeGenericFuncDesc) expression;
+    Class<?> op = expr.getGenericUDF().getClass();
+
+    // handle the logical operators
+    if (op == GenericUDFOPOr.class) {
+      builder.startOr();
+      addChildren(expr);
+      builder.end();
+    } else if (op == GenericUDFOPAnd.class) {
+      builder.startAnd();
+      addChildren(expr);
+      builder.end();
+    } else if (op == GenericUDFOPNot.class) {
+      builder.startNot();
+      addChildren(expr);
+      builder.end();
+    } else if (op == GenericUDFOPEqual.class) {
+      createLeaf(PredicateLeaf.Operator.EQUALS, expr);
+    } else if (op == GenericUDFOPNotEqual.class) {
+      builder.startNot();
+      createLeaf(PredicateLeaf.Operator.EQUALS, expr);
+      builder.end();
+    } else if (op == GenericUDFOPEqualNS.class) {
+      createLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS, expr);
+    } else if (op == GenericUDFOPGreaterThan.class) {
+      builder.startNot();
+      createLeaf(PredicateLeaf.Operator.LESS_THAN_EQUALS, expr);
+      builder.end();
+    } else if (op == GenericUDFOPEqualOrGreaterThan.class) {
+      builder.startNot();
+      createLeaf(PredicateLeaf.Operator.LESS_THAN, expr);
+      builder.end();
+    } else if (op == GenericUDFOPLessThan.class) {
+      createLeaf(PredicateLeaf.Operator.LESS_THAN, expr);
+    } else if (op == GenericUDFOPEqualOrLessThan.class) {
+      createLeaf(PredicateLeaf.Operator.LESS_THAN_EQUALS, expr);
+    } else if (op == GenericUDFIn.class) {
+      createLeaf(PredicateLeaf.Operator.IN, expr, 0);
+    } else if (op == GenericUDFBetween.class) {
+      createLeaf(PredicateLeaf.Operator.BETWEEN, expr, 1);
+    } else if (op == GenericUDFOPNull.class) {
+      createLeaf(PredicateLeaf.Operator.IS_NULL, expr, 0);
+    } else if (op == GenericUDFOPNotNull.class) {
+      builder.startNot();
+      createLeaf(PredicateLeaf.Operator.IS_NULL, expr, 0);
+      builder.end();
+
+      // otherwise, we didn't understand it, so mark it maybe
+    } else {
+      builder.literal(SearchArgument.TruthValue.YES_NO_NULL);
+    }
+  }
+
+
+  public static final String SARG_PUSHDOWN = "sarg.pushdown";
+
+  public static SearchArgument create(ExprNodeGenericFuncDesc expression) {
+    return new ConvertAstToSearchArg(expression).buildSearchArgument();
+  }
+
+
+  public static SearchArgument create(String kryo) {
+    Input input = new Input(Base64.decodeBase64(kryo));
+    return new Kryo().readObject(input, SearchArgumentImpl.class);
+  }
+
+  public static SearchArgument createFromConf(Configuration conf) {
+    String sargString;
+    if ((sargString = conf.get(TableScanDesc.FILTER_EXPR_CONF_STR)) != null) {
+      return create(Utilities.deserializeExpression(sargString));
+    } else if ((sargString = conf.get(SARG_PUSHDOWN)) != null) {
+      return create(sargString);
+    }
+    return null;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/c178a6e9/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentFactory.java
index c75e820..6ad927d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentFactory.java
@@ -18,6 +18,9 @@
 
 package org.apache.hadoop.hive.ql.io.sarg;
 
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.io.Input;
+import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
@@ -30,27 +33,7 @@ import org.apache.hadoop.hive.ql.plan.TableScanDesc;
  * A factory for creating SearchArguments.
  */
 public class SearchArgumentFactory {
-  public static final String SARG_PUSHDOWN = "sarg.pushdown";
-
-  public static SearchArgument create(ExprNodeGenericFuncDesc expression) {
-    return new SearchArgumentImpl(expression);
-  }
-
   public static Builder newBuilder() {
-    return SearchArgumentImpl.newBuilder();
-  }
-
-  public static SearchArgument create(String kryo) {
-    return SearchArgumentImpl.fromKryo(kryo);
-  }
-
-  public static SearchArgument createFromConf(Configuration conf) {
-    String sargString = null;
-    if ((sargString = conf.get(TableScanDesc.FILTER_EXPR_CONF_STR)) != null) {
-      return create(Utilities.deserializeExpression(sargString));
-    } else if ((sargString = conf.get(SARG_PUSHDOWN)) != null) {
-      return create(sargString);
-    }
-    return null;
+    return new SearchArgumentImpl.BuilderImpl();
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/c178a6e9/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentImpl.java
index 46f1e4e..1582a75 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentImpl.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentImpl.java
@@ -18,47 +18,20 @@
 
 package org.apache.hadoop.hive.ql.io.sarg;
 
-import java.math.BigDecimal;
 import java.sql.Timestamp;
 import java.util.ArrayDeque;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Deque;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
 import org.apache.commons.codec.binary.Base64;
-import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hive.common.type.HiveChar;
-import org.apache.hadoop.hive.common.type.HiveDecimal;
-import org.apache.hadoop.hive.common.type.HiveVarchar;
-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.ExprNodeGenericFuncDesc;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBetween;
-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.GenericUDFOPEqualNS;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrGreaterThan;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrLessThan;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPGreaterThan;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPLessThan;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNot;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNotEqual;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNotNull;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNull;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr;
-import org.apache.hadoop.hive.serde2.io.DateWritable;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
-import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 
 import com.esotericsoftware.kryo.Kryo;
-import com.esotericsoftware.kryo.io.Input;
 import com.esotericsoftware.kryo.io.Output;
 
 /**
@@ -74,6 +47,8 @@ final class SearchArgumentImpl implements SearchArgument {
     private final Object literal;
     private final List<Object> literalList;
 
+    // Used by kryo
+    @SuppressWarnings("unused")
     PredicateLeafImpl() {
       operator = null;
       type = null;
@@ -91,7 +66,24 @@ final class SearchArgumentImpl implements SearchArgument {
       this.type = type;
       this.columnName = columnName;
       this.literal = literal;
+      if (literal != null) {
+        if (literal.getClass() != type.getValueClass()) {
+          throw new IllegalArgumentException("Wrong value class " +
+              literal.getClass().getName() + " for " + type + "." + operator +
+              " leaf");
+        }
+      }
       this.literalList = literalList;
+      if (literalList != null) {
+        Class valueCls = type.getValueClass();
+        for(Object lit: literalList) {
+          if (lit != null && lit.getClass() != valueCls) {
+            throw new IllegalArgumentException("Wrong value class item " +
+                lit.getClass().getName() + " for " + type + "." + operator +
+                " leaf");
+          }
+        }
+      }
     }
 
     @Override
@@ -138,7 +130,7 @@ final class SearchArgumentImpl implements SearchArgument {
       } else if (literalList != null) {
         for(Object lit: literalList) {
           buffer.append(' ');
-          buffer.append(lit.toString());
+          buffer.append(lit == null ? "null" : lit.toString());
         }
       }
       buffer.append(')');
@@ -146,13 +138,9 @@ final class SearchArgumentImpl implements SearchArgument {
     }
 
     private static boolean isEqual(Object left, Object right) {
-      if (left == right) {
-        return true;
-      } else if (left == null || right == null) {
-        return false;
-      } else {
-        return left.equals(right);
-      }
+
+      return left == right ||
+          (left != null && right != null && left.equals(right));
     }
 
     @Override
@@ -182,286 +170,315 @@ final class SearchArgumentImpl implements SearchArgument {
     }
   }
 
-  static class ExpressionBuilder {
-    // max threshold for CNF conversion. having >8 elements in andList will be converted to maybe
+
+  private final List<PredicateLeaf> leaves;
+  private final ExpressionTree expression;
+
+  SearchArgumentImpl(ExpressionTree expression, List<PredicateLeaf> leaves) {
+    this.expression = expression;
+    this.leaves = leaves;
+  }
+
+  // Used by kyro
+  @SuppressWarnings("unused")
+  SearchArgumentImpl() {
+        leaves = null;
+        expression = null;
+  }
+
+  @Override
+  public List<PredicateLeaf> getLeaves() {
+    return leaves;
+  }
+
+  @Override
+  public TruthValue evaluate(TruthValue[] leaves) {
+    return expression == null ? TruthValue.YES : expression.evaluate(leaves);
+  }
+
+  @Override
+  public ExpressionTree getExpression() {
+    return expression;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder buffer = new StringBuilder();
+    for(int i=0; i < leaves.size(); ++i) {
+      buffer.append("leaf-");
+      buffer.append(i);
+      buffer.append(" = ");
+      buffer.append(leaves.get(i).toString());
+      buffer.append('\n');
+    }
+    buffer.append("expr = ");
+    buffer.append(expression);
+    return buffer.toString();
+  }
+
+  public String toKryo() {
+    Output out = new Output(4 * 1024, 10 * 1024 * 1024);
+    new Kryo().writeObject(out, this);
+    out.close();
+    return Base64.encodeBase64String(out.toBytes());
+  }
+
+  static class BuilderImpl implements Builder {
+
+    // max threshold for CNF conversion. having >8 elements in andList will be
+    // converted to maybe
     private static final int CNF_COMBINATIONS_THRESHOLD = 256;
-    private final List<PredicateLeaf> leaves = new ArrayList<PredicateLeaf>();
 
-    /**
-     * Get the type of the given expression node.
-     * @param expr the expression to get the type of
-     * @return int, string, or float or null if we don't know the type
-     */
-    private static PredicateLeaf.Type getType(ExprNodeDesc expr) {
-      TypeInfo type = expr.getTypeInfo();
-      if (type.getCategory() == ObjectInspector.Category.PRIMITIVE) {
-        switch (((PrimitiveTypeInfo) type).getPrimitiveCategory()) {
-          case BYTE:
-          case SHORT:
-          case INT:
-            return PredicateLeaf.Type.INTEGER;
-          case LONG:
-            return PredicateLeaf.Type.LONG;
-          case CHAR:
-          case VARCHAR:
-          case STRING:
-            return PredicateLeaf.Type.STRING;
-          case FLOAT:
-          case DOUBLE:
-            return PredicateLeaf.Type.FLOAT;
-          case DATE:
-            return PredicateLeaf.Type.DATE;
-          case TIMESTAMP:
-            return PredicateLeaf.Type.TIMESTAMP;
-          case DECIMAL:
-            return PredicateLeaf.Type.DECIMAL;
-          case BOOLEAN:
-        	return PredicateLeaf.Type.BOOLEAN;
-          default:
-        }
-      }
-      return null;
+    private final Deque<ExpressionTree> currentTree =
+        new ArrayDeque<ExpressionTree>();
+    private final Map<PredicateLeaf, Integer> leaves =
+        new HashMap<PredicateLeaf, Integer>();
+    private final ExpressionTree root =
+        new ExpressionTree(ExpressionTree.Operator.AND);
+    {
+      currentTree.add(root);
     }
 
-    /**
-     * Get the column name referenced in the expression. It must be at the top
-     * level of this expression and there must be exactly one column.
-     * @param expr the expression to look in
-     * @param variable the slot the variable is expected in
-     * @return the column name or null if there isn't exactly one column
-     */
-    private static String getColumnName(ExprNodeGenericFuncDesc expr,
-                                        int variable) {
-      List<ExprNodeDesc> children = expr.getChildren();
-      if (variable < 0 || variable >= children.size()) {
-        return null;
+    @Override
+    public Builder startOr() {
+      ExpressionTree node = new ExpressionTree(ExpressionTree.Operator.OR);
+      currentTree.getFirst().getChildren().add(node);
+      currentTree.addFirst(node);
+      return this;
+    }
+
+    @Override
+    public Builder startAnd() {
+      ExpressionTree node = new ExpressionTree(ExpressionTree.Operator.AND);
+      currentTree.getFirst().getChildren().add(node);
+      currentTree.addFirst(node);
+      return this;
+    }
+
+    @Override
+    public Builder startNot() {
+      ExpressionTree node = new ExpressionTree(ExpressionTree.Operator.NOT);
+      currentTree.getFirst().getChildren().add(node);
+      currentTree.addFirst(node);
+      return this;
+    }
+
+    @Override
+    public Builder end() {
+      ExpressionTree current = currentTree.removeFirst();
+      if (current.getChildren().size() == 0) {
+        throw new IllegalArgumentException("Can't create expression " + root +
+            " with no children.");
       }
-      ExprNodeDesc child = children.get(variable);
-      if (child instanceof ExprNodeColumnDesc) {
-        return ((ExprNodeColumnDesc) child).getColumn();
+      if (current.getOperator() == ExpressionTree.Operator.NOT &&
+          current.getChildren().size() != 1) {
+        throw new IllegalArgumentException("Can't create not expression " +
+            current + " with more than 1 child.");
       }
-      return null;
+      return this;
     }
 
-    private static Object boxLiteral(ExprNodeConstantDesc lit) {
-      switch (getType(lit)) {
-        case INTEGER:
-          return ((Number) lit.getValue()).intValue();
-        case LONG:
-          return ((Number) lit.getValue()).longValue();
-        case STRING:
-          return StringUtils.stripEnd(lit.getValue().toString(), null);
-        case FLOAT:
-          return Double.parseDouble(lit.getValue().toString());
-        case DATE:
-        case TIMESTAMP:
-        case DECIMAL:
-        case BOOLEAN:
-          return lit;
-        default:
-          throw new IllegalArgumentException("Unknown literal " + getType(lit));
+    private int addLeaf(PredicateLeaf leaf) {
+      Integer result = leaves.get(leaf);
+      if (result == null) {
+        int id = leaves.size();
+        leaves.put(leaf, id);
+        return id;
+      } else {
+        return result;
       }
     }
 
-    private static Object getLiteral(ExprNodeGenericFuncDesc expr) {
-      Object result = null;
-      List<ExprNodeDesc> children = expr.getChildren();
-      if (children.size() != 2) {
-        return null;
-      }
-      for(ExprNodeDesc child: children) {
-        if (child instanceof ExprNodeConstantDesc) {
-          if (result != null) {
-            return null;
-          }
-          result = boxLiteral((ExprNodeConstantDesc) child);
-        }
+    @Override
+    public Builder lessThan(String column, PredicateLeaf.Type type,
+                            Object literal) {
+      ExpressionTree parent = currentTree.getFirst();
+      if (column == null || literal == null) {
+        parent.getChildren().add(new ExpressionTree(TruthValue.YES_NO_NULL));
+      } else {
+        PredicateLeaf leaf =
+            new PredicateLeafImpl(PredicateLeaf.Operator.LESS_THAN,
+                type, column, literal, null);
+        parent.getChildren().add(new ExpressionTree(addLeaf(leaf)));
       }
-      return result;
+      return this;
     }
 
-    private static List<Object> getLiteralList(ExprNodeGenericFuncDesc expr,
-                                               int start) {
-      List<Object> result = new ArrayList<Object>();
-      List<ExprNodeDesc> children = expr.getChildren();
-      // ignore the first child, since it is the variable
-      for(ExprNodeDesc child: children.subList(start, children.size())) {
-        if (child instanceof ExprNodeConstantDesc) {
-          result.add(boxLiteral((ExprNodeConstantDesc) child));
-        } else {
-          // if we get some non-literals, we need to punt
-          return null;
-        }
+    @Override
+    public Builder lessThanEquals(String column, PredicateLeaf.Type type,
+                                  Object literal) {
+      ExpressionTree parent = currentTree.getFirst();
+      if (column == null || literal == null) {
+        parent.getChildren().add(new ExpressionTree(TruthValue.YES_NO_NULL));
+      } else {
+        PredicateLeaf leaf =
+            new PredicateLeafImpl(PredicateLeaf.Operator.LESS_THAN_EQUALS,
+                type, column, literal, null);
+        parent.getChildren().add(new ExpressionTree(addLeaf(leaf)));
       }
-      return result;
+      return this;
     }
 
-    private ExpressionTree createLeaf(PredicateLeaf.Operator operator,
-                                      ExprNodeGenericFuncDesc expression,
-                                      List<PredicateLeaf> leafCache,
-                                      int variable) {
-      String columnName = getColumnName(expression, variable);
-      if (columnName == null) {
-        return new ExpressionTree(TruthValue.YES_NO_NULL);
-      }
-      PredicateLeaf.Type type = getType(expression.getChildren().get(variable));
-      if (type == null) {
-        return new ExpressionTree(TruthValue.YES_NO_NULL);
+    @Override
+    public Builder equals(String column, PredicateLeaf.Type type,
+                          Object literal) {
+      ExpressionTree parent = currentTree.getFirst();
+      if (column == null || literal == null) {
+        parent.getChildren().add(new ExpressionTree(TruthValue.YES_NO_NULL));
+      } else {
+        PredicateLeaf leaf =
+            new PredicateLeafImpl(PredicateLeaf.Operator.EQUALS,
+                type, column, literal, null);
+        parent.getChildren().add(new ExpressionTree(addLeaf(leaf)));
       }
+      return this;
+    }
 
-      Object literal = null;
-      List<Object> literalList = null;
-      switch (operator) {
-        case IS_NULL:
-          break;
-        case IN:
-        case BETWEEN:
-          literalList = getLiteralList(expression, variable + 1);
-          if (literalList == null) {
-            return new ExpressionTree(TruthValue.YES_NO_NULL);
-          }
-          break;
-        default:
-          literal = getLiteral(expression);
-          if (literal == null) {
-            return new ExpressionTree(TruthValue.YES_NO_NULL);
-          }
-          break;
+    @Override
+    public Builder nullSafeEquals(String column, PredicateLeaf.Type type,
+                                  Object literal) {
+      ExpressionTree parent = currentTree.getFirst();
+      if (column == null || literal == null) {
+        parent.getChildren().add(new ExpressionTree(TruthValue.YES_NO_NULL));
+      } else {
+        PredicateLeaf leaf =
+            new PredicateLeafImpl(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+                type, column, literal, null);
+        parent.getChildren().add(new ExpressionTree(addLeaf(leaf)));
       }
-      // if the variable was on the right, we need to swap things around
-      boolean needSwap = false;
-      if (variable != 0) {
-        if (operator == PredicateLeaf.Operator.LESS_THAN) {
-          needSwap = true;
-          operator = PredicateLeaf.Operator.LESS_THAN_EQUALS;
-        } else if (operator == PredicateLeaf.Operator.LESS_THAN_EQUALS) {
-          needSwap = true;
-          operator = PredicateLeaf.Operator.LESS_THAN;
+      return this;
+    }
+
+    @Override
+    public Builder in(String column, PredicateLeaf.Type type,
+                      Object... literal) {
+      ExpressionTree parent = currentTree.getFirst();
+      if (column  == null || literal == null) {
+        parent.getChildren().add(new ExpressionTree(TruthValue.YES_NO_NULL));
+      } else {
+        if (literal.length == 0) {
+          throw new IllegalArgumentException("Can't create in expression with "
+              + "no arguments");
         }
+        List<Object> argList = new ArrayList<Object>();
+        argList.addAll(Arrays.asList(literal));
+
+        PredicateLeaf leaf =
+            new PredicateLeafImpl(PredicateLeaf.Operator.IN,
+                type, column, null, argList);
+        parent.getChildren().add(new ExpressionTree(addLeaf(leaf)));
       }
-      leafCache.add(new PredicateLeafImpl(operator, type, columnName,
-          literal, literalList));
-      ExpressionTree result = new ExpressionTree(leafCache.size() - 1);
-      if (needSwap) {
-        result = negate(result);
-      }
-      return result;
+      return this;
     }
 
-    /**
-     * Find the variable in the expression.
-     * @param expr the expression to look in
-     * @return the index of the variable or -1 if there is not exactly one
-     *   variable.
-     */
-    private int findVariable(ExprNodeDesc expr) {
-      int result = -1;
-      List<ExprNodeDesc> children = expr.getChildren();
-      for(int i = 0; i < children.size(); ++i) {
-        ExprNodeDesc child = children.get(i);
-        if (child instanceof ExprNodeColumnDesc) {
-          // if we already found a variable, this isn't a sarg
-          if (result != -1) {
-            return -1;
-          } else {
-            result = i;
-          }
-        }
+    @Override
+    public Builder isNull(String column, PredicateLeaf.Type type) {
+      ExpressionTree parent = currentTree.getFirst();
+      if (column == null) {
+        parent.getChildren().add(new ExpressionTree(TruthValue.YES_NO_NULL));
+      } else {
+        PredicateLeaf leaf =
+            new PredicateLeafImpl(PredicateLeaf.Operator.IS_NULL,
+                type, column, null, null);
+        parent.getChildren().add(new ExpressionTree(addLeaf(leaf)));
       }
-      return result;
+      return this;
     }
 
-    /**
-     * Create a leaf expression when we aren't sure where the variable is
-     * located.
-     * @param operator the operator type that was found
-     * @param expression the expression to check
-     * @param leafCache the list of leaves
-     * @return if the expression is a sarg, return it, otherwise null
-     */
-    private ExpressionTree createLeaf(PredicateLeaf.Operator operator,
-                                      ExprNodeGenericFuncDesc expression,
-                                      List<PredicateLeaf> leafCache) {
-      return createLeaf(operator, expression, leafCache,
-          findVariable(expression));
+    @Override
+    public Builder between(String column, PredicateLeaf.Type type, Object lower,
+                           Object upper) {
+      ExpressionTree parent = currentTree.getFirst();
+      if (column == null || lower == null || upper == null) {
+        parent.getChildren().add(new ExpressionTree(TruthValue.YES_NO_NULL));
+      } else {
+        List<Object> argList = new ArrayList<Object>();
+        argList.add(lower);
+        argList.add(upper);
+        PredicateLeaf leaf =
+            new PredicateLeafImpl(PredicateLeaf.Operator.BETWEEN,
+                type, column, null, argList);
+        parent.getChildren().add(new ExpressionTree(addLeaf(leaf)));
+      }
+      return this;
     }
 
-    private ExpressionTree negate(ExpressionTree expr) {
-      ExpressionTree result = new ExpressionTree(ExpressionTree.Operator.NOT);
-      result.getChildren().add(expr);
-      return result;
+    @Override
+    public Builder literal(TruthValue truth) {
+      ExpressionTree parent = currentTree.getFirst();
+      parent.getChildren().add(new ExpressionTree(truth));
+      return this;
     }
 
-    private void addChildren(ExpressionTree result,
-                             ExprNodeGenericFuncDesc node,
-                             List<PredicateLeaf> leafCache) {
-      for(ExprNodeDesc child: node.getChildren()) {
-        result.getChildren().add(parse(child, leafCache));
+    /**
+     * Recursively explore the tree to find the leaves that are still reachable
+     * after optimizations.
+     * @param tree the node to check next
+     * @param next the next available leaf id
+     * @param leafReorder
+     * @return the next available leaf id
+     */
+    static int compactLeaves(ExpressionTree tree, int next, int[] leafReorder) {
+      if (tree.getOperator() == ExpressionTree.Operator.LEAF) {
+        int oldLeaf = tree.getLeaf();
+        if (leafReorder[oldLeaf] == -1) {
+          leafReorder[oldLeaf] = next++;
+        }
+      } else if (tree.getChildren() != null){
+        for(ExpressionTree child: tree.getChildren()) {
+          next = compactLeaves(child, next, leafReorder);
+        }
       }
+      return next;
     }
 
     /**
-     * Do the recursive parse of the Hive ExprNodeDesc into our ExpressionTree.
-     * @param expression the Hive ExprNodeDesc
-     * @return the non-normalized ExpressionTree
+     * Rewrite expression tree to update the leaves.
+     * @param root the root of the tree to fix
+     * @param leafReorder a map from old leaf ids to new leaf ids
+     * @return the fixed root
      */
-    private ExpressionTree parse(ExprNodeDesc expression,
-                                 List<PredicateLeaf> leafCache) {
-      // if we don't know the expression, just assume maybe
-      if (expression.getClass() != ExprNodeGenericFuncDesc.class) {
-        return new ExpressionTree(TruthValue.YES_NO_NULL);
+    static ExpressionTree rewriteLeaves(ExpressionTree root,
+                                        int[] leafReorder) {
+      if (root.getOperator() == ExpressionTree.Operator.LEAF) {
+        return new ExpressionTree(leafReorder[root.getLeaf()]);
+      } else if (root.getChildren() != null){
+        List<ExpressionTree> children = root.getChildren();
+        for(int i=0; i < children.size(); ++i) {
+          children.set(i, rewriteLeaves(children.get(i), leafReorder));
+        }
       }
-      // get the kind of expression
-      ExprNodeGenericFuncDesc expr = (ExprNodeGenericFuncDesc) expression;
-      Class<?> op = expr.getGenericUDF().getClass();
-      ExpressionTree result;
-
-      // handle the logical operators
-      if (op == GenericUDFOPOr.class) {
-        result = new ExpressionTree(ExpressionTree.Operator.OR);
-        addChildren(result, expr, leafCache);
-      } else if (op == GenericUDFOPAnd.class) {
-        result = new ExpressionTree(ExpressionTree.Operator.AND);
-        addChildren(result, expr, leafCache);
-      } else if (op == GenericUDFOPNot.class) {
-        result = new ExpressionTree(ExpressionTree.Operator.NOT);
-        addChildren(result, expr, leafCache);
-      } else if (op == GenericUDFOPEqual.class) {
-        result = createLeaf(PredicateLeaf.Operator.EQUALS, expr, leafCache);
-      } else if (op == GenericUDFOPNotEqual.class) {
-        result = negate(createLeaf(PredicateLeaf.Operator.EQUALS, expr,
-            leafCache));
-      } else if (op == GenericUDFOPEqualNS.class) {
-        result = createLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS, expr,
-            leafCache);
-      } else if (op == GenericUDFOPGreaterThan.class) {
-        result = negate(createLeaf(PredicateLeaf.Operator.LESS_THAN_EQUALS,
-            expr, leafCache));
-      } else if (op == GenericUDFOPEqualOrGreaterThan.class) {
-        result = negate(createLeaf(PredicateLeaf.Operator.LESS_THAN, expr,
-            leafCache));
-      } else if (op == GenericUDFOPLessThan.class) {
-        result = createLeaf(PredicateLeaf.Operator.LESS_THAN, expr, leafCache);
-      } else if (op == GenericUDFOPEqualOrLessThan.class) {
-        result = createLeaf(PredicateLeaf.Operator.LESS_THAN_EQUALS, expr,
-            leafCache);
-      } else if (op == GenericUDFIn.class) {
-        result = createLeaf(PredicateLeaf.Operator.IN, expr, leafCache, 0);
-      } else if (op == GenericUDFBetween.class) {
-        result = createLeaf(PredicateLeaf.Operator.BETWEEN, expr, leafCache,
-            1);
-      } else if (op == GenericUDFOPNull.class) {
-        result = createLeaf(PredicateLeaf.Operator.IS_NULL, expr, leafCache,
-            0);
-      } else if (op == GenericUDFOPNotNull.class) {
-        result = negate(createLeaf(PredicateLeaf.Operator.IS_NULL, expr,
-            leafCache, 0));
+      return root;
+    }
 
-      // otherwise, we didn't understand it, so mark it maybe
-      } else {
-        result = new ExpressionTree(TruthValue.YES_NO_NULL);
+    @Override
+    public SearchArgument build() {
+      if (currentTree.size() != 1) {
+        throw new IllegalArgumentException("Failed to end " +
+            currentTree.size() + " operations.");
       }
-      return result;
+      ExpressionTree optimized = pushDownNot(root);
+      optimized = foldMaybe(optimized);
+      optimized = flatten(optimized);
+      optimized = convertToCNF(optimized);
+      optimized = flatten(optimized);
+      int leafReorder[] = new int[leaves.size()];
+      Arrays.fill(leafReorder, -1);
+      int newLeafCount = compactLeaves(optimized, 0, leafReorder);
+      optimized = rewriteLeaves(optimized, leafReorder);
+      ArrayList<PredicateLeaf> leafList = new ArrayList<>(newLeafCount);
+      // expand list to correct size
+      for(int i=0; i < newLeafCount; ++i) {
+        leafList.add(null);
+      }
+      // build the new list
+      for(Map.Entry<PredicateLeaf, Integer> elem: leaves.entrySet()) {
+        int newLoc = leafReorder[elem.getValue()];
+        if (newLoc != -1) {
+          leafList.set(newLoc, elem.getKey());
+        }
+      }
+      return new SearchArgumentImpl(optimized, leafList);
     }
 
     /**
@@ -528,7 +545,7 @@ final class SearchArgumentImpl implements SearchArgument {
                 return child;
               default:
                 throw new IllegalStateException("Got a maybe as child of " +
-                  expr);
+                    expr);
             }
           } else {
             expr.getChildren().set(i, child);
@@ -542,6 +559,45 @@ final class SearchArgumentImpl implements SearchArgument {
     }
 
     /**
+     * Converts multi-level ands and ors into single level ones.
+     * @param root the expression to flatten
+     * @return the flattened expression, which will always be root with
+     *   potentially modified children.
+     */
+    static ExpressionTree flatten(ExpressionTree root) {
+      if (root.getChildren() != null) {
+        // iterate through the index, so that if we add more children,
+        // they don't get re-visited
+        for(int i=0; i < root.getChildren().size(); ++i) {
+          ExpressionTree child = flatten(root.getChildren().get(i));
+          // do we need to flatten?
+          if (child.getOperator() == root.getOperator() &&
+              child.getOperator() != ExpressionTree.Operator.NOT) {
+            boolean first = true;
+            for(ExpressionTree grandkid: child.getChildren()) {
+              // for the first grandkid replace the original parent
+              if (first) {
+                first = false;
+                root.getChildren().set(i, grandkid);
+              } else {
+                root.getChildren().add(++i, grandkid);
+              }
+            }
+          } else {
+            root.getChildren().set(i, child);
+          }
+        }
+        // if we have a singleton AND or OR, just return the child
+        if ((root.getOperator() == ExpressionTree.Operator.OR ||
+            root.getOperator() == ExpressionTree.Operator.AND) &&
+            root.getChildren().size() == 1) {
+          return root.getChildren().get(0);
+        }
+      }
+      return root;
+    }
+
+    /**
      * Generate all combinations of items on the andList. For each item on the
      * andList, it generates all combinations of one child from each and
      * expression. Thus, (and a b) (and c d) will be expanded to: (or a c)
@@ -554,7 +610,7 @@ final class SearchArgumentImpl implements SearchArgument {
     private static void generateAllCombinations(List<ExpressionTree> result,
                                                 List<ExpressionTree> andList,
                                                 List<ExpressionTree> nonAndList
-                                               ) {
+    ) {
       List<ExpressionTree> kids = andList.get(0).getChildren();
       if (result.isEmpty()) {
         for(ExpressionTree kid: kids) {
@@ -637,391 +693,5 @@ final class SearchArgumentImpl implements SearchArgument {
       return true;
     }
 
-    /**
-     * Converts multi-level ands and ors into single level ones.
-     * @param root the expression to flatten
-     * @return the flattened expression, which will always be root with
-     *   potentially modified children.
-     */
-    static ExpressionTree flatten(ExpressionTree root) {
-      if (root.getChildren() != null) {
-        // iterate through the index, so that if we add more children,
-        // they don't get re-visited
-        for(int i=0; i < root.getChildren().size(); ++i) {
-          ExpressionTree child = flatten(root.getChildren().get(i));
-          // do we need to flatten?
-          if (child.getOperator() == root.getOperator() &&
-              child.getOperator() != ExpressionTree.Operator.NOT) {
-            boolean first = true;
-            for(ExpressionTree grandkid: child.getChildren()) {
-              // for the first grandkid replace the original parent
-              if (first) {
-                first = false;
-                root.getChildren().set(i, grandkid);
-              } else {
-                root.getChildren().add(++i, grandkid);
-              }
-            }
-          } else {
-            root.getChildren().set(i, child);
-          }
-        }
-        // if we have a singleton AND or OR, just return the child
-        if ((root.getOperator() == ExpressionTree.Operator.OR ||
-             root.getOperator() == ExpressionTree.Operator.AND) &&
-            root.getChildren().size() == 1) {
-          return root.getChildren().get(0);
-        }
-      }
-      return root;
-    }
-
-    /**
-     * Iterates through the expression, finding all of the leaves. It creates
-     * the leaves list with each unique leaf that is found in the expression.
-     * The expression is updated with the new leaf ids for each leaf.
-     * @param expr the expression to find the leaves in
-     * @param leafCache the list of all of the leaves
-     * @param lookup a map that is used to uniquify the leaves
-     * @return The potentially modified expression
-     */
-    private ExpressionTree buildLeafList(ExpressionTree expr,
-                                         List<PredicateLeaf> leafCache,
-                                         Map<PredicateLeaf,
-                                             ExpressionTree> lookup) {
-      if (expr.getChildren() != null) {
-        for(int i=0; i < expr.getChildren().size(); ++i) {
-          expr.getChildren().set(i, buildLeafList(expr.getChildren().get(i),
-              leafCache, lookup));
-        }
-      } else if (expr.getOperator() == ExpressionTree.Operator.LEAF) {
-        PredicateLeaf leaf = leafCache.get(expr.getLeaf());
-        ExpressionTree val = lookup.get(leaf);
-        if (val == null) {
-          val = new ExpressionTree(leaves.size());
-          lookup.put(leaf, val);
-          leaves.add(leaf);
-        }
-        return val;
-      }
-      return expr;
-    }
-
-    /**
-     * Builds the expression and leaf list from the original predicate.
-     * @param expression the expression to translate
-     * @return The normalized expression.
-     */
-    ExpressionTree expression(ExprNodeGenericFuncDesc expression) {
-      List<PredicateLeaf> leafCache = new ArrayList<PredicateLeaf>();
-      ExpressionTree expr = parse(expression, leafCache);
-      return expression(expr, leafCache);
-    }
-
-    /**
-     * Builds the expression and optimized leaf list from a non-normalized
-     * expression. Sets the leaves field with the unique leaves.
-     * @param expr non-normalized expression
-     * @param leaves non-unique leaves
-     * @return the normalized expression
-     */
-    ExpressionTree expression(ExpressionTree expr,
-                              List<PredicateLeaf> leaves) {
-      expr = pushDownNot(expr);
-      expr = foldMaybe(expr);
-      expr = flatten(expr);
-      expr = convertToCNF(expr);
-      expr = flatten(expr);
-      expr =  buildLeafList(expr, leaves,
-          new HashMap<PredicateLeaf, ExpressionTree>());
-      return expr;
-    }
-
-    List<PredicateLeaf> getLeaves() {
-      return leaves;
-    }
-  }
-
-  private final List<PredicateLeaf> leaves;
-  private final ExpressionTree expression;
-
-  SearchArgumentImpl(ExprNodeGenericFuncDesc expr) {
-    if (expr == null) {
-      leaves = new ArrayList<PredicateLeaf>();
-      expression = null;
-    } else {
-      ExpressionBuilder builder = new ExpressionBuilder();
-      expression = builder.expression(expr);
-      leaves = builder.getLeaves();
-    }
-  }
-
-  SearchArgumentImpl() {
-    leaves = null;
-    expression = null;
-  }
-
-  SearchArgumentImpl(ExpressionTree expression, List<PredicateLeaf> leaves) {
-    this.expression = expression;
-    this.leaves = leaves;
-  }
-
-  @Override
-  public List<PredicateLeaf> getLeaves() {
-    return leaves;
-  }
-
-  @Override
-  public TruthValue evaluate(TruthValue[] leaves) {
-    return expression == null ? TruthValue.YES : expression.evaluate(leaves);
-  }
-
-  @Override
-  public ExpressionTree getExpression() {
-    return expression;
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder buffer = new StringBuilder();
-    for(int i=0; i < leaves.size(); ++i) {
-      buffer.append("leaf-");
-      buffer.append(i);
-      buffer.append(" = ");
-      buffer.append(leaves.get(i).toString());
-      buffer.append('\n');
-    }
-    buffer.append("expr = ");
-    buffer.append(expression);
-    return buffer.toString();
-  }
-
-  public String toKryo() {
-    Output out = new Output(4 * 1024, 10 * 1024 * 1024);
-    new Kryo().writeObject(out, this);
-    out.close();
-    return Base64.encodeBase64String(out.toBytes());
-  }
-
-  static SearchArgument fromKryo(String value) {
-    Input input = new Input(Base64.decodeBase64(value));
-    return new Kryo().readObject(input, SearchArgumentImpl.class);
-  }
-
-  private static class BuilderImpl implements Builder {
-    private final Deque<ExpressionTree> currentTree =
-        new ArrayDeque<ExpressionTree>();
-    private final List<PredicateLeaf> leaves = new ArrayList<PredicateLeaf>();
-    private ExpressionTree root = null;
-
-    @Override
-    public Builder startOr() {
-      ExpressionTree node = new ExpressionTree(ExpressionTree.Operator.OR);
-      if (currentTree.size() != 0) {
-        ExpressionTree parent = currentTree.getFirst();
-        parent.getChildren().add(node);
-      }
-      currentTree.addFirst(node);
-      return this;
-    }
-
-    @Override
-    public Builder startAnd() {
-      ExpressionTree node = new ExpressionTree(ExpressionTree.Operator.AND);
-      if (currentTree.size() != 0) {
-        ExpressionTree parent = currentTree.getFirst();
-        parent.getChildren().add(node);
-      }
-      currentTree.addFirst(node);
-      return this;
-    }
-
-    @Override
-    public Builder startNot() {
-      ExpressionTree node = new ExpressionTree(ExpressionTree.Operator.NOT);
-      if (currentTree.size() != 0) {
-        ExpressionTree parent = currentTree.getFirst();
-        parent.getChildren().add(node);
-      }
-      currentTree.addFirst(node);
-      return this;
-    }
-
-    @Override
-    public Builder end() {
-      root = currentTree.removeFirst();
-      if (root.getChildren().size() == 0) {
-        throw new IllegalArgumentException("Can't create expression " + root +
-            " with no children.");
-      }
-      if (root.getOperator() == ExpressionTree.Operator.NOT &&
-          root.getChildren().size() != 1) {
-        throw new IllegalArgumentException("Can't create not expression " +
-            root + " with more than 1 child.");
-      }
-      return this;
-    }
-
-    private static Object boxLiteral(Object literal) {
-      if (literal instanceof String ||
-          literal instanceof Long ||
-          literal instanceof Double ||
-          literal instanceof DateWritable ||
-          literal instanceof Timestamp ||
-          literal instanceof HiveDecimal ||
-          literal instanceof BigDecimal ||
-          literal instanceof Boolean) {
-        return literal;
-      } else if (literal instanceof HiveChar ||
-          literal instanceof HiveVarchar) {
-        return StringUtils.stripEnd(literal.toString(), null);
-      } else if (literal instanceof Byte ||
-          literal instanceof Short ||
-          literal instanceof Integer) {
-        return ((Number) literal).longValue();
-      } else if (literal instanceof Float) {
-        // to avoid change in precision when upcasting float to double
-        // we convert the literal to string and parse it as double. (HIVE-8460)
-        return Double.parseDouble(literal.toString());
-      } else {
-        throw new IllegalArgumentException("Unknown type for literal " +
-            literal);
-      }
-    }
-
-    private static PredicateLeaf.Type getType(Object literal) {
-      if (literal instanceof Byte ||
-          literal instanceof Short ||
-          literal instanceof Integer) {
-        return PredicateLeaf.Type.INTEGER;
-      } else if(literal instanceof Long){
-        return PredicateLeaf.Type.LONG;
-      }else if (literal instanceof HiveChar ||
-          literal instanceof HiveVarchar ||
-          literal instanceof String) {
-        return PredicateLeaf.Type.STRING;
-      } else if (literal instanceof Float ||
-          literal instanceof Double) {
-        return PredicateLeaf.Type.FLOAT;
-      } else if (literal instanceof DateWritable) {
-        return PredicateLeaf.Type.DATE;
-      } else if (literal instanceof Timestamp) {
-        return PredicateLeaf.Type.TIMESTAMP;
-      }else if (literal instanceof HiveDecimal ||
-          literal instanceof BigDecimal) {
-        return PredicateLeaf.Type.DECIMAL;
-      } else if (literal instanceof Boolean) {
-        return PredicateLeaf.Type.BOOLEAN;
-      }
-      throw new IllegalArgumentException("Unknown type for literal " + literal);
-    }
-
-    @Override
-    public Builder lessThan(String column, Object literal) {
-      ExpressionTree parent = currentTree.getFirst();
-      Object box = boxLiteral(literal);
-      PredicateLeaf leaf =
-          new PredicateLeafImpl(PredicateLeaf.Operator.LESS_THAN,
-              getType(box), column, box, null);
-      leaves.add(leaf);
-      parent.getChildren().add(new ExpressionTree(leaves.size() - 1));
-      return this;
-    }
-
-    @Override
-    public Builder lessThanEquals(String column, Object literal) {
-      ExpressionTree parent = currentTree.getFirst();
-      Object box = boxLiteral(literal);
-      PredicateLeaf leaf =
-          new PredicateLeafImpl(PredicateLeaf.Operator.LESS_THAN_EQUALS,
-              getType(box), column, box, null);
-      leaves.add(leaf);
-      parent.getChildren().add(new ExpressionTree(leaves.size() - 1));
-      return this;
-    }
-
-    @Override
-    public Builder equals(String column, Object literal) {
-      ExpressionTree parent = currentTree.getFirst();
-      Object box = boxLiteral(literal);
-      PredicateLeaf leaf =
-          new PredicateLeafImpl(PredicateLeaf.Operator.EQUALS,
-              getType(box), column, box, null);
-      leaves.add(leaf);
-      parent.getChildren().add(new ExpressionTree(leaves.size() - 1));
-      return this;
-    }
-
-    @Override
-    public Builder nullSafeEquals(String column, Object literal) {
-      ExpressionTree parent = currentTree.getFirst();
-      Object box = boxLiteral(literal);
-      PredicateLeaf leaf =
-          new PredicateLeafImpl(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-              getType(box), column, box, null);
-      leaves.add(leaf);
-      parent.getChildren().add(new ExpressionTree(leaves.size() - 1));
-      return this;
-    }
-
-    @Override
-    public Builder in(String column, Object... literal) {
-      ExpressionTree parent = currentTree.getFirst();
-      if (literal.length == 0) {
-        throw new IllegalArgumentException("Can't create in expression with "
-            + "no arguments");
-      }
-      List<Object> argList = new ArrayList<Object>();
-      for(Object lit: literal){
-        argList.add(boxLiteral(lit));
-      }
-
-      PredicateLeaf leaf =
-          new PredicateLeafImpl(PredicateLeaf.Operator.IN,
-              getType(argList.get(0)), column, null, argList);
-      leaves.add(leaf);
-      parent.getChildren().add(new ExpressionTree(leaves.size() - 1));
-      return this;
-    }
-
-    @Override
-    public Builder isNull(String column) {
-      ExpressionTree parent = currentTree.getFirst();
-      PredicateLeaf leaf =
-          new PredicateLeafImpl(PredicateLeaf.Operator.IS_NULL,
-              PredicateLeaf.Type.STRING, column, null, null);
-      leaves.add(leaf);
-      parent.getChildren().add(new ExpressionTree(leaves.size() - 1));
-      return this;
-    }
-
-    @Override
-    public Builder between(String column, Object lower, Object upper) {
-      ExpressionTree parent = currentTree.getFirst();
-      List<Object> argList = new ArrayList<Object>();
-      argList.add(boxLiteral(lower));
-      argList.add(boxLiteral(upper));
-      PredicateLeaf leaf =
-          new PredicateLeafImpl(PredicateLeaf.Operator.BETWEEN,
-              getType(argList.get(0)), column, null, argList);
-      leaves.add(leaf);
-      parent.getChildren().add(new ExpressionTree(leaves.size() - 1));
-      return this;
-    }
-
-    @Override
-    public SearchArgument build() {
-      if (currentTree.size() != 0) {
-        throw new IllegalArgumentException("Failed to end " +
-            currentTree.size() + " operations.");
-      }
-      ExpressionBuilder internal = new ExpressionBuilder();
-      ExpressionTree normalized = internal.expression(root, leaves);
-      return new SearchArgumentImpl(normalized, internal.getLeaves());
-    }
-  }
-
-  public static Builder newBuilder() {
-    return new BuilderImpl();
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/c178a6e9/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
index e96ab2a..e40e1d2 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
@@ -66,6 +66,7 @@ import org.apache.hadoop.hive.ql.io.HiveInputFormat;
 import org.apache.hadoop.hive.ql.io.HiveOutputFormat;
 import org.apache.hadoop.hive.ql.io.InputFormatChecker;
 import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.SplitStrategy;
+import org.apache.hadoop.hive.ql.io.sarg.ConvertAstToSearchArg;
 import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory;
@@ -1746,8 +1747,8 @@ public class TestInputOutputFormat {
     types.add(builder.build());
     types.add(builder.build());
     SearchArgument isNull = SearchArgumentFactory.newBuilder()
-        .startAnd().isNull("cost").end().build();
-    conf.set(SearchArgumentFactory.SARG_PUSHDOWN, isNull.toKryo());
+        .startAnd().isNull("cost", PredicateLeaf.Type.INTEGER).end().build();
+    conf.set(ConvertAstToSearchArg.SARG_PUSHDOWN, isNull.toKryo());
     conf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR,
         "url,cost");
     options.include(new boolean[]{true, true, false, true, false});
@@ -1791,7 +1792,7 @@ public class TestInputOutputFormat {
     SearchArgument sarg =
         SearchArgumentFactory.newBuilder()
             .startAnd()
-            .lessThan("z", new Integer(0))
+            .lessThan("z", PredicateLeaf.Type.INTEGER, new Integer(0))
             .end()
             .build();
     conf.set("sarg.pushdown", sarg.toKryo());
@@ -1833,7 +1834,7 @@ public class TestInputOutputFormat {
     SearchArgument sarg =
         SearchArgumentFactory.newBuilder()
             .startAnd()
-            .lessThan("z", new String("foo"))
+            .lessThan("z", PredicateLeaf.Type.STRING, new String("foo"))
             .end()
             .build();
     conf.set("sarg.pushdown", sarg.toKryo());

http://git-wip-us.apache.org/repos/asf/hive/blob/c178a6e9/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcFile.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcFile.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcFile.java
index 255565e..4480d22 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcFile.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcFile.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.io.orc.OrcFile.Version;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory;
 import org.apache.hadoop.hive.serde2.io.ByteWritable;
@@ -1922,9 +1923,9 @@ public class TestOrcFile {
     SearchArgument sarg = SearchArgumentFactory.newBuilder()
         .startAnd()
           .startNot()
-             .lessThan("int1", 300000)
+             .lessThan("int1", PredicateLeaf.Type.INTEGER, 300000)
           .end()
-          .lessThan("int1", 600000)
+          .lessThan("int1", PredicateLeaf.Type.INTEGER, 600000)
         .end()
         .build();
     RecordReader rows = reader.rowsOptions(new Reader.Options()
@@ -1945,7 +1946,7 @@ public class TestOrcFile {
     // look through the file with no rows selected
     sarg = SearchArgumentFactory.newBuilder()
         .startAnd()
-          .lessThan("int1", 0)
+          .lessThan("int1", PredicateLeaf.Type.INTEGER, 0)
         .end()
         .build();
     rows = reader.rowsOptions(new Reader.Options()
@@ -1958,9 +1959,9 @@ public class TestOrcFile {
     // select first 100 and last 100 rows
     sarg = SearchArgumentFactory.newBuilder()
         .startOr()
-          .lessThan("int1", 300 * 100)
+          .lessThan("int1", PredicateLeaf.Type.INTEGER, 300 * 100)
           .startNot()
-            .lessThan("int1", 300 * 3400)
+            .lessThan("int1", PredicateLeaf.Type.INTEGER, 300 * 3400)
           .end()
         .end()
         .build();


[09/43] hive git commit: HIVE-11303: Getting Tez LimitExceededException after dag execution on large query (Jason Dere, reviewed by Vikram Dixit)

Posted by xu...@apache.org.
HIVE-11303: Getting Tez LimitExceededException after dag execution on large query (Jason Dere, reviewed by Vikram Dixit)


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

Branch: refs/heads/spark
Commit: 04d54f61c9f56906160936751e772080c079498c
Parents: 9904162
Author: Jason Dere <jd...@hortonworks.com>
Authored: Tue Jul 21 14:03:12 2015 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Tue Jul 21 14:03:12 2015 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   3 +
 .../test/resources/testconfiguration.properties |   4 +
 .../apache/hadoop/hive/ql/exec/JoinUtil.java    |  87 +-
 .../hadoop/hive/ql/exec/MapJoinOperator.java    |   2 +-
 .../apache/hadoop/hive/ql/exec/Operator.java    |   6 +
 .../hive/ql/exec/tez/KeyValuesAdapter.java      |  47 ++
 .../hive/ql/exec/tez/KeyValuesFromKeyValue.java |  90 +++
 .../ql/exec/tez/KeyValuesFromKeyValues.java     |  48 ++
 .../hive/ql/exec/tez/ReduceRecordProcessor.java |  11 +-
 .../hive/ql/exec/tez/ReduceRecordSource.java    |  12 +-
 .../ql/exec/vector/VectorMapJoinOperator.java   |   1 -
 .../mapjoin/VectorMapJoinCommonOperator.java    |   1 +
 .../hive/ql/optimizer/ConvertJoinMapJoin.java   | 213 +++--
 .../hive/ql/optimizer/MapJoinProcessor.java     |  44 +-
 .../ql/optimizer/ReduceSinkMapJoinProc.java     |  84 +-
 .../hadoop/hive/ql/parse/GenTezProcContext.java |  12 +
 .../hadoop/hive/ql/parse/GenTezUtils.java       |  23 +-
 .../apache/hadoop/hive/ql/parse/GenTezWork.java |  81 +-
 .../apache/hadoop/hive/ql/plan/BaseWork.java    |   2 +-
 .../hive/ql/plan/CommonMergeJoinDesc.java       |   4 +
 .../hadoop/hive/ql/plan/ExprNodeDescUtils.java  | 115 +++
 .../apache/hadoop/hive/ql/plan/MapJoinDesc.java |  11 +
 .../apache/hadoop/hive/ql/plan/ReduceWork.java  |   2 +-
 .../clientpositive/tez_dynpart_hashjoin_1.q     | 101 +++
 .../clientpositive/tez_dynpart_hashjoin_2.q     |  83 ++
 .../tez_vector_dynpart_hashjoin_1.q             | 102 +++
 .../tez_vector_dynpart_hashjoin_2.q             |  84 ++
 .../tez/tez_dynpart_hashjoin_1.q.out            | 791 ++++++++++++++++++
 .../tez/tez_dynpart_hashjoin_2.q.out            | 564 +++++++++++++
 .../tez/tez_vector_dynpart_hashjoin_1.q.out     | 804 +++++++++++++++++++
 .../tez/tez_vector_dynpart_hashjoin_2.q.out     | 570 +++++++++++++
 31 files changed, 3899 insertions(+), 103 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/04d54f61/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 39477d6..33b67dd 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -1046,6 +1046,9 @@ public class HiveConf extends Configuration {
         "job, process those skewed keys. The same key need not be skewed for all the tables, and so,\n" +
         "the follow-up map-reduce job (for the skewed keys) would be much faster, since it would be a\n" +
         "map-join."),
+    HIVEDYNAMICPARTITIONHASHJOIN("hive.optimize.dynamic.partition.hashjoin", false,
+        "Whether to enable dynamically partitioned hash join optimization. \n" +
+        "This setting is also dependent on enabling hive.auto.convert.join"),
     HIVECONVERTJOIN("hive.auto.convert.join", true,
         "Whether Hive enables the optimization about converting common join into mapjoin based on the input file size"),
     HIVECONVERTJOINNOCONDITIONALTASK("hive.auto.convert.join.noconditionaltask", true,

http://git-wip-us.apache.org/repos/asf/hive/blob/04d54f61/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index 97715fc..fbde465 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -334,6 +334,10 @@ minitez.query.files=bucket_map_join_tez1.q,\
   tez_dml.q,\
   tez_fsstat.q,\
   tez_insert_overwrite_local_directory_1.q,\
+  tez_dynpart_hashjoin_1.q,\
+  tez_dynpart_hashjoin_2.q,\
+  tez_vector_dynpart_hashjoin_1.q,\
+  tez_vector_dynpart_hashjoin_2.q,\
   tez_join_hash.q,\
   tez_join_result_complex.q,\
   tez_join_tests.q,\

http://git-wip-us.apache.org/repos/asf/hive/blob/04d54f61/ql/src/java/org/apache/hadoop/hive/ql/exec/JoinUtil.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/JoinUtil.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/JoinUtil.java
index 7b57550..0aaa51a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/JoinUtil.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/JoinUtil.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
 import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 import org.apache.hadoop.mapred.Reporter;
@@ -65,8 +66,21 @@ public class JoinUtil {
 
     int iterate = Math.min(exprEntries.length, inputObjInspector.length);
     for (byte alias = 0; alias < iterate; alias++) {
+      ObjectInspector inputOI = inputObjInspector[alias];
+
+      // For vectorized reduce-side operators getting inputs from a reduce sink,
+      // the row object inspector will get a flattened version of the object inspector
+      // where the nested key/value structs are replaced with a single struct:
+      // Example: { key: { reducesinkkey0:int }, value: { _col0:int, _col1:int, .. } }
+      // Would get converted to the following for a vectorized input:
+      //   { 'key.reducesinkkey0':int, 'value._col0':int, 'value._col1':int, .. }
+      // The ExprNodeEvaluator initialzation below gets broken with the flattened
+      // object inpsectors, so convert it back to the a form that contains the
+      // nested key/value structs.
+      inputOI = unflattenObjInspector(inputOI);
+
       if (alias == (byte) posBigTableAlias ||
-          exprEntries[alias] == null || inputObjInspector[alias] == null) {
+          exprEntries[alias] == null || inputOI == null) {
         // skip the driver and directly loadable tables
         continue;
       }
@@ -74,7 +88,7 @@ public class JoinUtil {
       List<ExprNodeEvaluator> exprList = exprEntries[alias];
       List<ObjectInspector> fieldOIList = new ArrayList<ObjectInspector>();
       for (int i = 0; i < exprList.size(); i++) {
-        fieldOIList.add(exprList.get(i).initialize(inputObjInspector[alias]));
+        fieldOIList.add(exprList.get(i).initialize(inputOI));
       }
       result[alias] = fieldOIList;
     }
@@ -350,4 +364,73 @@ public class JoinUtil {
     rc.setTableDesc(tblDesc);
     return rc;
   }
+
+  private static String KEY_FIELD_PREFIX = (Utilities.ReduceField.KEY + ".").toLowerCase();
+  private static String VALUE_FIELD_PREFIX = (Utilities.ReduceField.VALUE + ".").toLowerCase();
+
+  /**
+   * Create a new struct object inspector for the list of struct fields, first removing the
+   * prefix from the field name.
+   * @param fields
+   * @param prefixToRemove
+   * @return
+   */
+  private static ObjectInspector createStructFromFields(List<StructField> fields, String prefixToRemove) {
+    int prefixLength = prefixToRemove.length() + 1; // also remove the '.' after the prefix
+    ArrayList<String> fieldNames = new ArrayList<String>();
+    ArrayList<ObjectInspector> fieldOIs = new ArrayList<ObjectInspector>();
+    for (StructField field : fields) {
+      fieldNames.add(field.getFieldName().substring(prefixLength));
+      fieldOIs.add(field.getFieldObjectInspector());
+    }
+    return ObjectInspectorFactory.getStandardStructObjectInspector(fieldNames, fieldOIs);
+  }
+
+  /**
+   * Checks the input object inspector to see if it is in for form of a flattened struct
+   * like the ones generated by a vectorized reduce sink input:
+   *   { 'key.reducesinkkey0':int, 'value._col0':int, 'value._col1':int, .. }
+   * If so, then it creates an "unflattened" struct that contains nested key/value
+   * structs:
+   *   { key: { reducesinkkey0:int }, value: { _col0:int, _col1:int, .. } }
+   *
+   * @param oi
+   * @return unflattened object inspector if unflattening is needed,
+   *         otherwise the original object inspector
+   */
+  private static ObjectInspector unflattenObjInspector(ObjectInspector oi) {
+    if (oi instanceof StructObjectInspector) {
+      // Check if all fields start with "key." or "value."
+      // If so, then unflatten by adding an additional level of nested key and value structs
+      // Example: { "key.reducesinkkey0":int, "key.reducesinkkey1": int, "value._col6":int }
+      // Becomes
+      //   { "key": { "reducesinkkey0":int, "reducesinkkey1":int }, "value": { "_col6":int } }
+      ArrayList<StructField> keyFields = new ArrayList<StructField>();
+      ArrayList<StructField> valueFields = new ArrayList<StructField>();
+      for (StructField field : ((StructObjectInspector) oi).getAllStructFieldRefs()) {
+        String fieldNameLower = field.getFieldName().toLowerCase();
+        if (fieldNameLower.startsWith(KEY_FIELD_PREFIX)) {
+          keyFields.add(field);
+        } else if (fieldNameLower.startsWith(VALUE_FIELD_PREFIX)) {
+          valueFields.add(field);
+        } else {
+          // Not a flattened struct, no need to unflatten
+          return oi;
+        }
+      }
+
+      // All field names are of the form "key." or "value."
+      // Create key/value structs and add the respective fields to each one
+      ArrayList<ObjectInspector> reduceFieldOIs = new ArrayList<ObjectInspector>();
+      reduceFieldOIs.add(createStructFromFields(keyFields, Utilities.ReduceField.KEY.toString()));
+      reduceFieldOIs.add(createStructFromFields(valueFields, Utilities.ReduceField.VALUE.toString()));
+
+      // Finally create the outer struct to contain the key, value structs
+      return ObjectInspectorFactory.getStandardStructObjectInspector(
+          Utilities.reduceFieldNameList,
+          reduceFieldOIs);
+    }
+
+    return oi;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/04d54f61/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java
index a40f0a9..1b9d7ef 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java
@@ -154,7 +154,7 @@ public class MapJoinOperator extends AbstractMapJoinOperator<MapJoinDesc> implem
     final ExecMapperContext mapContext = getExecContext();
     final MapredContext mrContext = MapredContext.get();
 
-    if (!conf.isBucketMapJoin()) {
+    if (!conf.isBucketMapJoin() && !conf.isDynamicPartitionHashJoin()) {
       /*
        * The issue with caching in case of bucket map join is that different tasks
        * process different buckets and if the container is reused to join a different bucket,

http://git-wip-us.apache.org/repos/asf/hive/blob/04d54f61/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java
index d7f1b42..0f02737 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java
@@ -1354,4 +1354,10 @@ public abstract class Operator<T extends OperatorDesc> implements Serializable,C
       return childOperators;
     }
   }
+
+  public void removeParents() {
+    for (Operator<?> parent : new ArrayList<Operator<?>>(getParentOperators())) {
+      removeParent(parent);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/04d54f61/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/KeyValuesAdapter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/KeyValuesAdapter.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/KeyValuesAdapter.java
new file mode 100644
index 0000000..8f706fe
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/KeyValuesAdapter.java
@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.tez;
+
+import java.io.IOException;
+
+/**
+ * Key-values interface for the Reader used by ReduceRecordSource
+ */
+public interface KeyValuesAdapter {
+  /**
+   * Get the key for current record
+   * @return
+   * @throws IOException
+   */
+  Object getCurrentKey() throws IOException;
+
+  /**
+   * Get the values for the current record
+   * @return
+   * @throws IOException
+   */
+  Iterable<Object> getCurrentValues() throws IOException;
+
+  /**
+   * Move to the next record
+   * @return true if successful, false if there are no more records to process
+   * @throws IOException
+   */
+  boolean next() throws IOException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/04d54f61/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/KeyValuesFromKeyValue.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/KeyValuesFromKeyValue.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/KeyValuesFromKeyValue.java
new file mode 100644
index 0000000..51cdeca
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/KeyValuesFromKeyValue.java
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.tez;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import org.apache.tez.runtime.library.api.KeyValueReader;
+
+/**
+ * Provides a key/values (note the plural values) interface out of a KeyValueReader,
+ * needed by ReduceRecordSource when reading input from a key/value source.
+ */
+public class KeyValuesFromKeyValue implements KeyValuesAdapter {
+  protected KeyValueReader reader;
+  protected ValueIterator<Object> valueIterator =
+      new ValueIterator<Object>();
+
+  private static class ValueIterator<T> implements Iterator<T>, Iterable<T> {
+
+    protected boolean hasNextValue = false;
+    protected T value = null;
+
+    @Override
+    public boolean hasNext() {
+      return hasNextValue;
+    }
+
+    @Override
+    public T next() {
+      if (!hasNextValue) {
+        throw new NoSuchElementException();
+      }
+      hasNextValue = false;
+      return value;
+    }
+
+    void reset(T value) {
+      this.value = value;
+      hasNextValue = true;
+    }
+
+    @Override
+    public void remove() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Iterator<T> iterator() {
+      return this;
+    }
+  }
+
+  public KeyValuesFromKeyValue(KeyValueReader reader) {
+    this.reader = reader;
+  }
+
+  @Override
+  public Object getCurrentKey() throws IOException {
+    return reader.getCurrentKey();
+  }
+
+  @Override
+  public Iterable<Object> getCurrentValues() throws IOException {
+    Object obj = reader.getCurrentValue();
+    valueIterator.reset(obj);
+    return valueIterator;
+  }
+
+  @Override
+  public boolean next() throws IOException {
+    return reader.next();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/04d54f61/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/KeyValuesFromKeyValues.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/KeyValuesFromKeyValues.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/KeyValuesFromKeyValues.java
new file mode 100644
index 0000000..b027bce
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/KeyValuesFromKeyValues.java
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.tez;
+
+import java.io.IOException;
+import org.apache.tez.runtime.library.api.KeyValuesReader;
+
+/**
+ * Provides a key/values interface out of a KeyValuesReader for use by ReduceRecordSource.
+ */
+public class KeyValuesFromKeyValues implements KeyValuesAdapter {
+  protected KeyValuesReader reader;
+
+  public KeyValuesFromKeyValues(KeyValuesReader reader) {
+    this.reader = reader;
+  }
+
+  @Override
+  public Object getCurrentKey() throws IOException {
+    return reader.getCurrentKey();
+  }
+
+  @Override
+  public Iterable<Object> getCurrentValues() throws IOException {
+    return reader.getCurrentValues();
+  }
+
+  @Override
+  public boolean next() throws IOException {
+    return reader.next();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/04d54f61/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/ReduceRecordProcessor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/ReduceRecordProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/ReduceRecordProcessor.java
index 545d7c6..d649672 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/ReduceRecordProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/ReduceRecordProcessor.java
@@ -50,6 +50,7 @@ import org.apache.tez.runtime.api.Input;
 import org.apache.tez.runtime.api.LogicalInput;
 import org.apache.tez.runtime.api.LogicalOutput;
 import org.apache.tez.runtime.api.ProcessorContext;
+import org.apache.tez.runtime.api.Reader;
 import org.apache.tez.runtime.library.api.KeyValuesReader;
 
 /**
@@ -129,10 +130,11 @@ public class ReduceRecordProcessor  extends RecordProcessor{
         tagToReducerMap.put(mergeReduceWork.getTag(), mergeReduceWork);
       }
 
-      bigTablePosition = (byte) reduceWork.getTag();
       ((TezContext) MapredContext.get()).setDummyOpsMap(connectOps);
     }
 
+    bigTablePosition = (byte) reduceWork.getTag();
+
     ObjectInspector[] mainWorkOIs = null;
     ((TezContext) MapredContext.get()).setInputs(inputs);
     ((TezContext) MapredContext.get()).setTezProcessorContext(processorContext);
@@ -227,10 +229,13 @@ public class ReduceRecordProcessor  extends RecordProcessor{
     reducer.setParentOperators(null); // clear out any parents as reducer is the root
 
     TableDesc keyTableDesc = redWork.getKeyDesc();
-    KeyValuesReader reader = (KeyValuesReader) inputs.get(inputName).getReader();
+    Reader reader = inputs.get(inputName).getReader();
 
     sources[tag] = new ReduceRecordSource();
-    sources[tag].init(jconf, redWork.getReducer(), redWork.getVectorMode(), keyTableDesc,
+    // Only the big table input source should be vectorized (if applicable)
+    // Note this behavior may have to change if we ever implement a vectorized merge join
+    boolean vectorizedRecordSource = (tag == bigTablePosition) && redWork.getVectorMode();
+    sources[tag].init(jconf, redWork.getReducer(), vectorizedRecordSource, keyTableDesc,
         valueTableDesc, reader, tag == bigTablePosition, (byte) tag,
         redWork.getVectorScratchColumnTypeMap());
     ois[tag] = sources[tag].getObjectInspector();

http://git-wip-us.apache.org/repos/asf/hive/blob/04d54f61/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/ReduceRecordSource.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/ReduceRecordSource.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/ReduceRecordSource.java
index 20f6dba..89f7572 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/ReduceRecordSource.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/ReduceRecordSource.java
@@ -57,6 +57,8 @@ import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.tez.runtime.api.Reader;
+import org.apache.tez.runtime.library.api.KeyValueReader;
 import org.apache.tez.runtime.library.api.KeyValuesReader;
 
 /**
@@ -107,7 +109,7 @@ public class ReduceRecordSource implements RecordSource {
   /* this is only used in the error code path */
   private List<VectorExpressionWriter> valueStringWriters;
 
-  private KeyValuesReader reader;
+  private KeyValuesAdapter reader;
 
   private boolean handleGroupKey;
 
@@ -120,7 +122,7 @@ public class ReduceRecordSource implements RecordSource {
   private final GroupIterator groupIterator = new GroupIterator();
 
   void init(JobConf jconf, Operator<?> reducer, boolean vectorized, TableDesc keyTableDesc,
-      TableDesc valueTableDesc, KeyValuesReader reader, boolean handleGroupKey, byte tag,
+      TableDesc valueTableDesc, Reader reader, boolean handleGroupKey, byte tag,
       Map<Integer, String> vectorScratchColumnTypeMap)
       throws Exception {
 
@@ -129,7 +131,11 @@ public class ReduceRecordSource implements RecordSource {
     this.reducer = reducer;
     this.vectorized = vectorized;
     this.keyTableDesc = keyTableDesc;
-    this.reader = reader;
+    if (reader instanceof KeyValueReader) {
+      this.reader = new KeyValuesFromKeyValue((KeyValueReader) reader);
+    } else {
+      this.reader = new KeyValuesFromKeyValues((KeyValuesReader) reader);
+    }
     this.handleGroupKey = handleGroupKey;
     this.tag = tag;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/04d54f61/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapJoinOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapJoinOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapJoinOperator.java
index e9bd44a..9bd811c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapJoinOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapJoinOperator.java
@@ -100,7 +100,6 @@ public class VectorMapJoinOperator extends VectorMapJoinBaseOperator {
 
   @Override
   public Collection<Future<?>> initializeOp(Configuration hconf) throws HiveException {
-
     // Use a final variable to properly parameterize the processVectorInspector closure.
     // Using a member variable in the closure will not do the right thing...
     final int parameterizePosBigTable = conf.getPosBigTable();

http://git-wip-us.apache.org/repos/asf/hive/blob/04d54f61/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinCommonOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinCommonOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinCommonOperator.java
index 4c8c4b1..87ebcf2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinCommonOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinCommonOperator.java
@@ -551,6 +551,7 @@ public abstract class VectorMapJoinCommonOperator extends MapJoinOperator implem
 
   @Override
   protected Collection<Future<?>> initializeOp(Configuration hconf) throws HiveException {
+
     Collection<Future<?>> result = super.initializeOp(hconf);
 
     if (LOG.isDebugEnabled()) {

http://git-wip-us.apache.org/repos/asf/hive/blob/04d54f61/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
index 5a87bd6..e3acdfc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
@@ -93,9 +93,7 @@ public class ConvertJoinMapJoin implements NodeProcessor {
       if (retval == null) {
         return retval;
       } else {
-        int pos = 0; // it doesn't matter which position we use in this case.
-        convertJoinSMBJoin(joinOp, context, pos, 0, false);
-        return null;
+        fallbackToReduceSideJoin(joinOp, context);
       }
     }
 
@@ -103,27 +101,8 @@ public class ConvertJoinMapJoin implements NodeProcessor {
     // exact number of buckets. Else choose the largest number of estimated
     // reducers from the parent operators.
     int numBuckets = -1;
-    int estimatedBuckets = -1;
     if (context.conf.getBoolVar(HiveConf.ConfVars.HIVE_CONVERT_JOIN_BUCKET_MAPJOIN_TEZ)) {
-      for (Operator<? extends OperatorDesc>parentOp : joinOp.getParentOperators()) {
-        if (parentOp.getOpTraits().getNumBuckets() > 0) {
-          numBuckets = (numBuckets < parentOp.getOpTraits().getNumBuckets()) ?
-              parentOp.getOpTraits().getNumBuckets() : numBuckets;
-        }
-
-        if (parentOp instanceof ReduceSinkOperator) {
-          ReduceSinkOperator rs = (ReduceSinkOperator) parentOp;
-          estimatedBuckets = (estimatedBuckets < rs.getConf().getNumReducers()) ?
-              rs.getConf().getNumReducers() : estimatedBuckets;
-        }
-      }
-
-      if (numBuckets <= 0) {
-        numBuckets = estimatedBuckets;
-        if (numBuckets <= 0) {
-          numBuckets = 1;
-        }
-      }
+      numBuckets = estimateNumBuckets(joinOp, true);
     } else {
       numBuckets = 1;
     }
@@ -136,7 +115,7 @@ public class ConvertJoinMapJoin implements NodeProcessor {
       } else {
         // only case is full outer join with SMB enabled which is not possible. Convert to regular
         // join.
-        convertJoinSMBJoin(joinOp, context, 0, 0, false);
+        fallbackToReduceSideJoin(joinOp, context);
         return null;
       }
     }
@@ -155,20 +134,18 @@ public class ConvertJoinMapJoin implements NodeProcessor {
     if (mapJoinConversionPos < 0) {
       // we are just converting to a common merge join operator. The shuffle
       // join in map-reduce case.
-      int pos = 0; // it doesn't matter which position we use in this case.
-      convertJoinSMBJoin(joinOp, context, pos, 0, false);
+      fallbackToReduceSideJoin(joinOp, context);
       return null;
     }
 
-    MapJoinOperator mapJoinOp = convertJoinMapJoin(joinOp, context, mapJoinConversionPos);
+    MapJoinOperator mapJoinOp = convertJoinMapJoin(joinOp, context, mapJoinConversionPos, true);
     // map join operator by default has no bucket cols and num of reduce sinks
     // reduced by 1
-    mapJoinOp
-.setOpTraits(new OpTraits(null, -1, null));
+    mapJoinOp.setOpTraits(new OpTraits(null, -1, null));
     mapJoinOp.setStatistics(joinOp.getStatistics());
     // propagate this change till the next RS
     for (Operator<? extends OperatorDesc> childOp : mapJoinOp.getChildOperators()) {
-      setAllChildrenTraitsToNull(childOp);
+      setAllChildrenTraits(childOp, mapJoinOp.getOpTraits());
     }
 
     return null;
@@ -180,7 +157,7 @@ public class ConvertJoinMapJoin implements NodeProcessor {
     // we cannot convert to bucket map join, we cannot convert to
     // map join either based on the size. Check if we can convert to SMB join.
     if (context.conf.getBoolVar(HiveConf.ConfVars.HIVE_AUTO_SORTMERGE_JOIN) == false) {
-      convertJoinSMBJoin(joinOp, context, 0, 0, false);
+      fallbackToReduceSideJoin(joinOp, context);
       return null;
     }
     Class<? extends BigTableSelectorForAutoSMJ> bigTableMatcherClass = null;
@@ -209,8 +186,7 @@ public class ConvertJoinMapJoin implements NodeProcessor {
       // contains aliases from sub-query
       // we are just converting to a common merge join operator. The shuffle
       // join in map-reduce case.
-      int pos = 0; // it doesn't matter which position we use in this case.
-      convertJoinSMBJoin(joinOp, context, pos, 0, false);
+      fallbackToReduceSideJoin(joinOp, context);
       return null;
     }
 
@@ -220,8 +196,7 @@ public class ConvertJoinMapJoin implements NodeProcessor {
     } else {
       // we are just converting to a common merge join operator. The shuffle
       // join in map-reduce case.
-      int pos = 0; // it doesn't matter which position we use in this case.
-      convertJoinSMBJoin(joinOp, context, pos, 0, false);
+      fallbackToReduceSideJoin(joinOp, context);
     }
     return null;
   }
@@ -317,16 +292,16 @@ public class ConvertJoinMapJoin implements NodeProcessor {
     mergeJoinOp.cloneOriginalParentsList(mergeJoinOp.getParentOperators());
   }
 
-  private void setAllChildrenTraitsToNull(Operator<? extends OperatorDesc> currentOp) {
+  private void setAllChildrenTraits(Operator<? extends OperatorDesc> currentOp, OpTraits opTraits) {
     if (currentOp instanceof ReduceSinkOperator) {
       return;
     }
-    currentOp.setOpTraits(new OpTraits(null, -1, null));
+    currentOp.setOpTraits(new OpTraits(opTraits.getBucketColNames(), opTraits.getNumBuckets(), opTraits.getSortCols()));
     for (Operator<? extends OperatorDesc> childOp : currentOp.getChildOperators()) {
       if ((childOp instanceof ReduceSinkOperator) || (childOp instanceof GroupByOperator)) {
         break;
       }
-      setAllChildrenTraitsToNull(childOp);
+      setAllChildrenTraits(childOp, opTraits);
     }
   }
 
@@ -338,7 +313,7 @@ public class ConvertJoinMapJoin implements NodeProcessor {
       return false;
     }
 
-    MapJoinOperator mapJoinOp = convertJoinMapJoin(joinOp, context, bigTablePosition);
+    MapJoinOperator mapJoinOp = convertJoinMapJoin(joinOp, context, bigTablePosition, true);
     MapJoinDesc joinDesc = mapJoinOp.getConf();
     joinDesc.setBucketMapJoin(true);
 
@@ -633,7 +608,7 @@ public class ConvertJoinMapJoin implements NodeProcessor {
    */
 
   public MapJoinOperator convertJoinMapJoin(JoinOperator joinOp, OptimizeTezProcContext context,
-      int bigTablePosition) throws SemanticException {
+      int bigTablePosition, boolean removeReduceSink) throws SemanticException {
     // bail on mux operator because currently the mux operator masks the emit keys
     // of the constituent reduce sinks.
     for (Operator<? extends OperatorDesc> parentOp : joinOp.getParentOperators()) {
@@ -646,45 +621,49 @@ public class ConvertJoinMapJoin implements NodeProcessor {
     MapJoinOperator mapJoinOp =
         MapJoinProcessor.convertJoinOpMapJoinOp(context.conf, joinOp,
             joinOp.getConf().isLeftInputJoin(), joinOp.getConf().getBaseSrc(),
-            joinOp.getConf().getMapAliases(), bigTablePosition, true);
+            joinOp.getConf().getMapAliases(), bigTablePosition, true, removeReduceSink);
     mapJoinOp.getConf().setHybridHashJoin(HiveConf.getBoolVar(context.conf,
-      HiveConf.ConfVars.HIVEUSEHYBRIDGRACEHASHJOIN));
+        HiveConf.ConfVars.HIVEUSEHYBRIDGRACEHASHJOIN));
 
     Operator<? extends OperatorDesc> parentBigTableOp =
         mapJoinOp.getParentOperators().get(bigTablePosition);
     if (parentBigTableOp instanceof ReduceSinkOperator) {
-      for (Operator<?> p : parentBigTableOp.getParentOperators()) {
-        // we might have generated a dynamic partition operator chain. Since
-        // we're removing the reduce sink we need do remove that too.
-        Set<Operator<?>> dynamicPartitionOperators = new HashSet<Operator<?>>();
-        Map<Operator<?>, AppMasterEventOperator> opEventPairs = new HashMap<>();
-        for (Operator<?> c : p.getChildOperators()) {
-          AppMasterEventOperator event = findDynamicPartitionBroadcast(c);
-          if (event != null) {
-            dynamicPartitionOperators.add(c);
-            opEventPairs.put(c, event);
+      if (removeReduceSink) {
+        for (Operator<?> p : parentBigTableOp.getParentOperators()) {
+          // we might have generated a dynamic partition operator chain. Since
+          // we're removing the reduce sink we need do remove that too.
+          Set<Operator<?>> dynamicPartitionOperators = new HashSet<Operator<?>>();
+          Map<Operator<?>, AppMasterEventOperator> opEventPairs = new HashMap<>();
+          for (Operator<?> c : p.getChildOperators()) {
+            AppMasterEventOperator event = findDynamicPartitionBroadcast(c);
+            if (event != null) {
+              dynamicPartitionOperators.add(c);
+              opEventPairs.put(c, event);
+            }
           }
-        }
-        for (Operator<?> c : dynamicPartitionOperators) {
-          if (context.pruningOpsRemovedByPriorOpt.isEmpty() ||
-              !context.pruningOpsRemovedByPriorOpt.contains(opEventPairs.get(c))) {
-            p.removeChild(c);
-            // at this point we've found the fork in the op pipeline that has the pruning as a child plan.
-            LOG.info("Disabling dynamic pruning for: "
-                + ((DynamicPruningEventDesc) opEventPairs.get(c).getConf()).getTableScan().getName()
-                + ". Need to be removed together with reduce sink");
+          for (Operator<?> c : dynamicPartitionOperators) {
+            if (context.pruningOpsRemovedByPriorOpt.isEmpty() ||
+                !context.pruningOpsRemovedByPriorOpt.contains(opEventPairs.get(c))) {
+              p.removeChild(c);
+              // at this point we've found the fork in the op pipeline that has the pruning as a child plan.
+              LOG.info("Disabling dynamic pruning for: "
+                  + ((DynamicPruningEventDesc) opEventPairs.get(c).getConf()).getTableScan().getName()
+                  + ". Need to be removed together with reduce sink");
+            }
+          }
+          for (Operator<?> op : dynamicPartitionOperators) {
+            context.pruningOpsRemovedByPriorOpt.add(opEventPairs.get(op));
           }
         }
-        for (Operator<?> op : dynamicPartitionOperators) {
-          context.pruningOpsRemovedByPriorOpt.add(opEventPairs.get(op));
+
+        mapJoinOp.getParentOperators().remove(bigTablePosition);
+        if (!(mapJoinOp.getParentOperators().contains(parentBigTableOp.getParentOperators().get(0)))) {
+          mapJoinOp.getParentOperators().add(bigTablePosition,
+              parentBigTableOp.getParentOperators().get(0));
         }
+        parentBigTableOp.getParentOperators().get(0).removeChild(parentBigTableOp);
       }
-      mapJoinOp.getParentOperators().remove(bigTablePosition);
-      if (!(mapJoinOp.getParentOperators().contains(parentBigTableOp.getParentOperators().get(0)))) {
-        mapJoinOp.getParentOperators().add(bigTablePosition,
-            parentBigTableOp.getParentOperators().get(0));
-      }
-      parentBigTableOp.getParentOperators().get(0).removeChild(parentBigTableOp);
+
       for (Operator<? extends OperatorDesc>op : mapJoinOp.getParentOperators()) {
         if (!(op.getChildOperators().contains(mapJoinOp))) {
           op.getChildOperators().add(mapJoinOp);
@@ -720,4 +699,100 @@ public class ConvertJoinMapJoin implements NodeProcessor {
 
     return null;
   }
+
+  /**
+   * Estimate the number of buckets in the join, using the parent operators' OpTraits and/or
+   * parent operators' number of reducers
+   * @param joinOp
+   * @param useOpTraits  Whether OpTraits should be used for the estimate.
+   * @return
+   */
+  private static int estimateNumBuckets(JoinOperator joinOp, boolean useOpTraits) {
+    int numBuckets = -1;
+    int estimatedBuckets = -1;
+
+    for (Operator<? extends OperatorDesc>parentOp : joinOp.getParentOperators()) {
+      if (parentOp.getOpTraits().getNumBuckets() > 0) {
+        numBuckets = (numBuckets < parentOp.getOpTraits().getNumBuckets()) ?
+            parentOp.getOpTraits().getNumBuckets() : numBuckets;
+      }
+
+      if (parentOp instanceof ReduceSinkOperator) {
+        ReduceSinkOperator rs = (ReduceSinkOperator) parentOp;
+        estimatedBuckets = (estimatedBuckets < rs.getConf().getNumReducers()) ?
+            rs.getConf().getNumReducers() : estimatedBuckets;
+      }
+    }
+
+    if (!useOpTraits) {
+      // Ignore the value we got from OpTraits.
+      // The logic below will fall back to the estimate from numReducers
+      numBuckets = -1;
+    }
+
+    if (numBuckets <= 0) {
+      numBuckets = estimatedBuckets;
+      if (numBuckets <= 0) {
+        numBuckets = 1;
+      }
+    }
+
+    return numBuckets;
+  }
+
+  private boolean convertJoinDynamicPartitionedHashJoin(JoinOperator joinOp, OptimizeTezProcContext context)
+    throws SemanticException {
+    // Attempt dynamic partitioned hash join
+    // Since we don't have big table index yet, must start with estimate of numReducers
+    int numReducers = estimateNumBuckets(joinOp, false);
+    LOG.info("Try dynamic partitioned hash join with estimated " + numReducers + " reducers");
+    int bigTablePos = getMapJoinConversionPos(joinOp, context, numReducers);
+    if (bigTablePos >= 0) {
+      // Now that we have the big table index, get real numReducers value based on big table RS
+      ReduceSinkOperator bigTableParentRS =
+          (ReduceSinkOperator) (joinOp.getParentOperators().get(bigTablePos));
+      numReducers = bigTableParentRS.getConf().getNumReducers();
+      LOG.debug("Real big table reducers = " + numReducers);
+
+      MapJoinOperator mapJoinOp = convertJoinMapJoin(joinOp, context, bigTablePos, false);
+      if (mapJoinOp != null) {
+        LOG.info("Selected dynamic partitioned hash join");
+        mapJoinOp.getConf().setDynamicPartitionHashJoin(true);
+        // Set OpTraits for dynamically partitioned hash join:
+        // bucketColNames: Re-use previous joinOp's bucketColNames. Parent operators should be
+        //   reduce sink, which should have bucket columns based on the join keys.
+        // numBuckets: set to number of reducers
+        // sortCols: This is an unsorted join - no sort cols
+        OpTraits opTraits = new OpTraits(
+            joinOp.getOpTraits().getBucketColNames(),
+            numReducers,
+            null);
+        mapJoinOp.setOpTraits(opTraits);
+        mapJoinOp.setStatistics(joinOp.getStatistics());
+        // propagate this change till the next RS
+        for (Operator<? extends OperatorDesc> childOp : mapJoinOp.getChildOperators()) {
+          setAllChildrenTraits(childOp, mapJoinOp.getOpTraits());
+        }
+        return true;
+      }
+    }
+
+    return false;
+  }
+
+  private void fallbackToReduceSideJoin(JoinOperator joinOp, OptimizeTezProcContext context)
+      throws SemanticException {
+    if (context.conf.getBoolVar(HiveConf.ConfVars.HIVECONVERTJOIN) &&
+        context.conf.getBoolVar(HiveConf.ConfVars.HIVEDYNAMICPARTITIONHASHJOIN)) {
+      if (convertJoinDynamicPartitionedHashJoin(joinOp, context)) {
+        return;
+      }
+    }
+
+    // we are just converting to a common merge join operator. The shuffle
+    // join in map-reduce case.
+    int pos = 0; // it doesn't matter which position we use in this case.
+    LOG.info("Fallback to common merge join operator");
+    convertJoinSMBJoin(joinOp, context, pos, 0, false);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/04d54f61/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java
index 4d84f0f..f8f2b7b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java
@@ -29,6 +29,8 @@ import java.util.Map;
 import java.util.Set;
 import java.util.Stack;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.ObjectPair;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -57,6 +59,7 @@ import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.lib.Rule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.parse.GenMapRedWalker;
+import org.apache.hadoop.hive.ql.parse.OptimizeTezProcContext;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
@@ -89,6 +92,7 @@ public class MapJoinProcessor implements Transform {
   // (column type + column name). The column name is not really used anywhere, but it
   // needs to be passed. Use the string defined below for that.
   private static final String MAPJOINKEY_FIELDPREFIX = "mapjoinkey";
+  private static final Log LOG = LogFactory.getLog(MapJoinProcessor.class.getName());
 
   public MapJoinProcessor() {
   }
@@ -356,11 +360,18 @@ public class MapJoinProcessor implements Transform {
   public static MapJoinOperator convertJoinOpMapJoinOp(HiveConf hconf,
       JoinOperator op, boolean leftInputJoin, String[] baseSrc, List<String> mapAliases,
       int mapJoinPos, boolean noCheckOuterJoin) throws SemanticException {
+    return convertJoinOpMapJoinOp(hconf, op, leftInputJoin, baseSrc, mapAliases,
+        mapJoinPos, noCheckOuterJoin, true);
+  }
+
+  public static MapJoinOperator convertJoinOpMapJoinOp(HiveConf hconf,
+      JoinOperator op, boolean leftInputJoin, String[] baseSrc, List<String> mapAliases,
+      int mapJoinPos, boolean noCheckOuterJoin, boolean adjustParentsChildren)
+          throws SemanticException {
 
     MapJoinDesc mapJoinDescriptor =
         getMapJoinDesc(hconf, op, leftInputJoin, baseSrc, mapAliases,
-                mapJoinPos, noCheckOuterJoin);
-
+            mapJoinPos, noCheckOuterJoin, adjustParentsChildren);
     // reduce sink row resolver used to generate map join op
     RowSchema outputRS = op.getSchema();
 
@@ -1025,7 +1036,7 @@ public class MapJoinProcessor implements Transform {
 
   public static MapJoinDesc getMapJoinDesc(HiveConf hconf,
       JoinOperator op, boolean leftInputJoin, String[] baseSrc, List<String> mapAliases,
-      int mapJoinPos, boolean noCheckOuterJoin) throws SemanticException {
+      int mapJoinPos, boolean noCheckOuterJoin, boolean adjustParentsChildren) throws SemanticException {
     JoinDesc desc = op.getConf();
     JoinCondDesc[] condns = desc.getConds();
     Byte[] tagOrder = desc.getTagOrder();
@@ -1072,6 +1083,26 @@ public class MapJoinProcessor implements Transform {
     // get the join keys from old parent ReduceSink operators
     Map<Byte, List<ExprNodeDesc>> keyExprMap = pair.getSecond();
 
+    if (!adjustParentsChildren) {
+      // Since we did not remove reduce sink parents, keep the original value expressions
+      newValueExprs = valueExprs;
+
+      // Join key exprs are represented in terms of the original table columns,
+      // we need to convert these to the generated column names we can see in the Join operator
+      Map<Byte, List<ExprNodeDesc>> newKeyExprMap = new HashMap<Byte, List<ExprNodeDesc>>();
+      for (Map.Entry<Byte, List<ExprNodeDesc>> mapEntry : keyExprMap.entrySet()) {
+        Byte pos = mapEntry.getKey();
+        ReduceSinkOperator rsParent = oldReduceSinkParentOps.get(pos.byteValue());
+        List<ExprNodeDesc> keyExprList =
+            ExprNodeDescUtils.resolveJoinKeysAsRSColumns(mapEntry.getValue(), rsParent);
+        if (keyExprList == null) {
+          throw new SemanticException("Error resolving join keys");
+        }
+        newKeyExprMap.put(pos, keyExprList);
+      }
+      keyExprMap = newKeyExprMap;
+    }
+
     // construct valueTableDescs and valueFilteredTableDescs
     List<TableDesc> valueTableDescs = new ArrayList<TableDesc>();
     List<TableDesc> valueFilteredTableDescs = new ArrayList<TableDesc>();
@@ -1163,4 +1194,11 @@ public class MapJoinProcessor implements Transform {
 
     return mapJoinDescriptor;
   }
+
+  public static MapJoinDesc getMapJoinDesc(HiveConf hconf,
+      JoinOperator op, boolean leftInputJoin, String[] baseSrc, List<String> mapAliases,
+      int mapJoinPos, boolean noCheckOuterJoin) throws SemanticException {
+    return getMapJoinDesc(hconf, op, leftInputJoin, baseSrc,
+        mapAliases, mapJoinPos, noCheckOuterJoin, true);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/04d54f61/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java
index bca91dd..b546838 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java
@@ -21,12 +21,15 @@ package org.apache.hadoop.hive.ql.optimizer;
 import java.util.ArrayList;
 import java.util.EnumSet;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.Stack;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.exec.HashTableDummyOperator;
 import org.apache.hadoop.hive.ql.exec.MapJoinOperator;
 import org.apache.hadoop.hive.ql.exec.Operator;
@@ -58,11 +61,13 @@ import org.apache.hadoop.hive.ql.plan.TezWork;
 import org.apache.hadoop.hive.ql.plan.TezWork.VertexType;
 import org.apache.hadoop.hive.ql.stats.StatsUtils;
 
+import com.google.common.collect.Sets;
+
 import static org.apache.hadoop.hive.ql.plan.ReduceSinkDesc.ReducerTraits.FIXED;
 
 public class ReduceSinkMapJoinProc implements NodeProcessor {
 
-  protected transient Log LOG = LogFactory.getLog(this.getClass().getName());
+  private final static Log LOG = LogFactory.getLog(ReduceSinkMapJoinProc.class.getName());
 
   /* (non-Javadoc)
    * This processor addresses the RS-MJ case that occurs in tez on the small/hash
@@ -79,7 +84,40 @@ public class ReduceSinkMapJoinProc implements NodeProcessor {
     GenTezProcContext context = (GenTezProcContext) procContext;
     MapJoinOperator mapJoinOp = (MapJoinOperator)nd;
 
-    if (stack.size() < 2 || !(stack.get(stack.size() - 2) instanceof ReduceSinkOperator)) {
+    // remember the original parent list before we start modifying it.
+    if (!context.mapJoinParentMap.containsKey(mapJoinOp)) {
+      List<Operator<?>> parents = new ArrayList<Operator<?>>(mapJoinOp.getParentOperators());
+      context.mapJoinParentMap.put(mapJoinOp, parents);
+    }
+
+    boolean isBigTable = stack.size() < 2
+        || !(stack.get(stack.size() - 2) instanceof ReduceSinkOperator);
+
+    ReduceSinkOperator parentRS = null;
+    if (!isBigTable) {
+      parentRS = (ReduceSinkOperator)stack.get(stack.size() - 2);
+
+      // For dynamic partitioned hash join, the big table will also be coming from a ReduceSinkOperator
+      // Check for this condition.
+      // TODO: use indexOf(), or parentRS.getTag()?
+      isBigTable =
+          (mapJoinOp.getParentOperators().indexOf(parentRS) == mapJoinOp.getConf().getPosBigTable());
+    }
+
+    if (mapJoinOp.getConf().isDynamicPartitionHashJoin() &&
+        !context.mapJoinToUnprocessedSmallTableReduceSinks.containsKey(mapJoinOp)) {
+      // Initialize set of unprocessed small tables
+      Set<ReduceSinkOperator> rsSet = Sets.newIdentityHashSet();
+      for (int pos = 0; pos < mapJoinOp.getParentOperators().size(); ++pos) {
+        if (pos == mapJoinOp.getConf().getPosBigTable()) {
+          continue;
+        }
+        rsSet.add((ReduceSinkOperator) mapJoinOp.getParentOperators().get(pos));
+      }
+      context.mapJoinToUnprocessedSmallTableReduceSinks.put(mapJoinOp, rsSet);
+    }
+
+    if (isBigTable) {
       context.currentMapJoinOperators.add(mapJoinOp);
       return null;
     }
@@ -87,14 +125,29 @@ public class ReduceSinkMapJoinProc implements NodeProcessor {
     context.preceedingWork = null;
     context.currentRootOperator = null;
 
-    ReduceSinkOperator parentRS = (ReduceSinkOperator)stack.get(stack.size() - 2);
+    return processReduceSinkToHashJoin(parentRS, mapJoinOp, context);
+  }
+
+  public static BaseWork getMapJoinParentWork(GenTezProcContext context, Operator<?> parentRS) {
+    BaseWork parentWork;
+    if (context.unionWorkMap.containsKey(parentRS)) {
+      parentWork = context.unionWorkMap.get(parentRS);
+    } else {
+      assert context.childToWorkMap.get(parentRS).size() == 1;
+      parentWork = context.childToWorkMap.get(parentRS).get(0);
+    }
+    return parentWork;
+  }
+
+  public static Object processReduceSinkToHashJoin(ReduceSinkOperator parentRS, MapJoinOperator mapJoinOp,
+      GenTezProcContext context) throws SemanticException {
     // remove the tag for in-memory side of mapjoin
     parentRS.getConf().setSkipTag(true);
     parentRS.setSkipTag(true);
-    // remember the original parent list before we start modifying it.
-    if (!context.mapJoinParentMap.containsKey(mapJoinOp)) {
-      List<Operator<?>> parents = new ArrayList<Operator<?>>(mapJoinOp.getParentOperators());
-      context.mapJoinParentMap.put(mapJoinOp, parents);
+
+    // Mark this small table as being processed
+    if (mapJoinOp.getConf().isDynamicPartitionHashJoin()) {
+      context.mapJoinToUnprocessedSmallTableReduceSinks.get(mapJoinOp).remove(parentRS);
     }
 
     List<BaseWork> mapJoinWork = null;
@@ -109,13 +162,7 @@ public class ReduceSinkMapJoinProc implements NodeProcessor {
      *
      */
     mapJoinWork = context.mapJoinWorkMap.get(mapJoinOp);
-    BaseWork parentWork;
-    if (context.unionWorkMap.containsKey(parentRS)) {
-      parentWork = context.unionWorkMap.get(parentRS);
-    } else {
-      assert context.childToWorkMap.get(parentRS).size() == 1;
-      parentWork = context.childToWorkMap.get(parentRS).get(0);
-    }
+    BaseWork parentWork = getMapJoinParentWork(context, parentRS);
 
     // set the link between mapjoin and parent vertex
     int pos = context.mapJoinParentMap.get(mapJoinOp).indexOf(parentRS);
@@ -161,6 +208,11 @@ public class ReduceSinkMapJoinProc implements NodeProcessor {
           keyCount /= bucketCount;
           tableSize /= bucketCount;
         }
+      } else if (joinConf.isDynamicPartitionHashJoin()) {
+        // For dynamic partitioned hash join, assuming table is split evenly among the reduce tasks.
+        bucketCount = parentRS.getConf().getNumReducers();
+        keyCount /= bucketCount;
+        tableSize /= bucketCount;
       }
     }
     LOG.info("Mapjoin " + mapJoinOp + ", pos: " + pos + " --> " + parentWork.getName() + " ("
@@ -218,6 +270,8 @@ public class ReduceSinkMapJoinProc implements NodeProcessor {
           edgeType = EdgeType.CUSTOM_SIMPLE_EDGE;
         }
       }
+    } else if (mapJoinOp.getConf().isDynamicPartitionHashJoin()) {
+      edgeType = EdgeType.CUSTOM_SIMPLE_EDGE;
     }
     TezEdgeProperty edgeProp = new TezEdgeProperty(null, edgeType, numBuckets);
 
@@ -232,7 +286,7 @@ public class ReduceSinkMapJoinProc implements NodeProcessor {
         }
 
         ReduceSinkOperator r = null;
-        if (parentRS.getConf().getOutputName() != null) {
+        if (context.connectedReduceSinks.contains(parentRS)) {
           LOG.debug("Cloning reduce sink for multi-child broadcast edge");
           // we've already set this one up. Need to clone for the next work.
           r = (ReduceSinkOperator) OperatorFactory.getAndMakeChild(

http://git-wip-us.apache.org/repos/asf/hive/blob/04d54f61/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java
index f474eae..9334c73 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezProcContext.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hive.ql.parse;
 
 import java.io.Serializable;
+import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
 import java.util.LinkedList;
@@ -107,6 +108,10 @@ public class GenTezProcContext implements NodeProcessorCtx{
   // map that says which mapjoin belongs to which work item
   public final Map<MapJoinOperator, List<BaseWork>> mapJoinWorkMap;
 
+  // Mapping of reducesink to mapjoin operators
+  // Only used for dynamic partitioned hash joins (mapjoin operator in the reducer)
+  public final Map<Operator<?>, MapJoinOperator> smallTableParentToMapJoinMap;
+
   // a map to keep track of which root generated which work
   public final Map<Operator<?>, BaseWork> rootToWorkMap;
 
@@ -151,6 +156,11 @@ public class GenTezProcContext implements NodeProcessorCtx{
   // remember the connections between ts and event
   public final Map<TableScanOperator, List<AppMasterEventOperator>> tsToEventMap;
 
+  // When processing dynamic partitioned hash joins, some of the small tables may not get processed
+  // before the mapjoin's parents are removed during GenTezWork.process(). This is to keep
+  // track of which small tables haven't been processed yet.
+  public Map<MapJoinOperator, Set<ReduceSinkOperator>> mapJoinToUnprocessedSmallTableReduceSinks;
+
   @SuppressWarnings("unchecked")
   public GenTezProcContext(HiveConf conf, ParseContext parseContext,
       List<Task<MoveWork>> moveTask, List<Task<? extends Serializable>> rootTasks,
@@ -167,6 +177,7 @@ public class GenTezProcContext implements NodeProcessorCtx{
     this.leafOperatorToFollowingWork = new LinkedHashMap<Operator<?>, BaseWork>();
     this.linkOpWithWorkMap = new LinkedHashMap<Operator<?>, Map<BaseWork, TezEdgeProperty>>();
     this.linkWorkWithReduceSinkMap = new LinkedHashMap<BaseWork, List<ReduceSinkOperator>>();
+    this.smallTableParentToMapJoinMap = new LinkedHashMap<Operator<?>, MapJoinOperator>();
     this.mapJoinWorkMap = new LinkedHashMap<MapJoinOperator, List<BaseWork>>();
     this.rootToWorkMap = new LinkedHashMap<Operator<?>, BaseWork>();
     this.childToWorkMap = new LinkedHashMap<Operator<?>, List<BaseWork>>();
@@ -188,6 +199,7 @@ public class GenTezProcContext implements NodeProcessorCtx{
     this.tsToEventMap = new LinkedHashMap<TableScanOperator, List<AppMasterEventOperator>>();
     this.opMergeJoinWorkMap = new LinkedHashMap<Operator<?>, MergeJoinWork>();
     this.currentMergeJoinOperator = null;
+    this.mapJoinToUnprocessedSmallTableReduceSinks = new HashMap<MapJoinOperator, Set<ReduceSinkOperator>>();
 
     rootTasks.add(currentTask);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/04d54f61/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezUtils.java
index 93ad145..a9d1f8e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezUtils.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hive.ql.exec.AppMasterEventOperator;
 import org.apache.hadoop.hive.ql.exec.FetchTask;
 import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
 import org.apache.hadoop.hive.ql.exec.HashTableDummyOperator;
+import org.apache.hadoop.hive.ql.exec.MapJoinOperator;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.OperatorUtils;
 import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
@@ -130,12 +131,13 @@ public class GenTezUtils {
     tezWork.add(reduceWork);
 
     TezEdgeProperty edgeProp;
+    EdgeType edgeType = determineEdgeType(context.preceedingWork, reduceWork);
     if (reduceWork.isAutoReduceParallelism()) {
       edgeProp =
-          new TezEdgeProperty(context.conf, EdgeType.SIMPLE_EDGE, true,
+          new TezEdgeProperty(context.conf, edgeType, true,
               reduceWork.getMinReduceTasks(), reduceWork.getMaxReduceTasks(), bytesPerReducer);
     } else {
-      edgeProp = new TezEdgeProperty(EdgeType.SIMPLE_EDGE);
+      edgeProp = new TezEdgeProperty(edgeType);
     }
 
     tezWork.connect(
@@ -470,4 +472,21 @@ public class GenTezUtils {
 
     curr.removeChild(child);
   }
+
+  public static EdgeType determineEdgeType(BaseWork preceedingWork, BaseWork followingWork) {
+    if (followingWork instanceof ReduceWork) {
+      // Ideally there should be a better way to determine that the followingWork contains
+      // a dynamic partitioned hash join, but in some cases (createReduceWork()) it looks like
+      // the work must be created/connected first, before the GenTezProcContext can be updated
+      // with the mapjoin/work relationship.
+      ReduceWork reduceWork = (ReduceWork) followingWork;
+      if (reduceWork.getReducer() instanceof MapJoinOperator) {
+        MapJoinOperator joinOp = (MapJoinOperator) reduceWork.getReducer();
+        if (joinOp.getConf().isDynamicPartitionHashJoin()) {
+          return EdgeType.CUSTOM_SIMPLE_EDGE;
+        }
+      }
+    }
+    return EdgeType.SIMPLE_EDGE;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/04d54f61/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezWork.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezWork.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezWork.java
index 6b3e19d..c4e0413 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezWork.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezWork.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hive.ql.lib.Node;
 import org.apache.hadoop.hive.ql.lib.NodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.optimizer.GenMapRedUtils;
+import org.apache.hadoop.hive.ql.optimizer.ReduceSinkMapJoinProc;
 import org.apache.hadoop.hive.ql.plan.BaseWork;
 import org.apache.hadoop.hive.ql.plan.MergeJoinWork;
 import org.apache.hadoop.hive.ql.plan.OperatorDesc;
@@ -165,8 +166,11 @@ public class GenTezWork implements NodeProcessor {
       mergeJoinWork.addMergedWork(work, null, context.leafOperatorToFollowingWork);
       Operator<? extends OperatorDesc> parentOp =
           getParentFromStack(context.currentMergeJoinOperator, stack);
+      // Set the big table position. Both the reduce work and merge join operator
+      // should be set with the same value.
       int pos = context.currentMergeJoinOperator.getTagForOperator(parentOp);
       work.setTag(pos);
+      context.currentMergeJoinOperator.getConf().setBigTablePosition(pos);
       tezWork.setVertexType(work, VertexType.MULTI_INPUT_UNINITIALIZED_EDGES);
       for (BaseWork parentWork : tezWork.getParents(work)) {
         TezEdgeProperty edgeProp = tezWork.getEdgeProperty(parentWork, work);
@@ -190,6 +194,50 @@ public class GenTezWork implements NodeProcessor {
     // remember which mapjoin operator links with which work
     if (!context.currentMapJoinOperators.isEmpty()) {
       for (MapJoinOperator mj: context.currentMapJoinOperators) {
+        // For dynamic partitioned hash join, ReduceSinkMapJoinProc rule may not get run for all
+        // of the ReduceSink parents, because the parents of the MapJoin operator get
+        // removed later on in this method. Keep track of the parent to mapjoin mapping
+        // so we can later run the same logic that is run in ReduceSinkMapJoinProc.
+        if (mj.getConf().isDynamicPartitionHashJoin()) {
+          // Since this is a dynamic partitioned hash join, the work for this join should be a ReduceWork
+          ReduceWork reduceWork = (ReduceWork) work;
+          int bigTablePosition = mj.getConf().getPosBigTable();
+          reduceWork.setTag(bigTablePosition);
+
+          // Use context.mapJoinParentMap to get the original RS parents, because
+          // the MapJoin's parents may have been replaced by dummy operator.
+          List<Operator<?>> mapJoinOriginalParents = context.mapJoinParentMap.get(mj);
+          if (mapJoinOriginalParents == null) {
+            throw new SemanticException("Unexpected error - context.mapJoinParentMap did not have an entry for " + mj);
+          }
+          for (int pos = 0; pos < mapJoinOriginalParents.size(); ++pos) {
+            // This processing only needs to happen for the small tables
+            if (pos == bigTablePosition) {
+              continue;
+            }
+            Operator<?> parentOp = mapJoinOriginalParents.get(pos);
+            context.smallTableParentToMapJoinMap.put(parentOp, mj);
+
+            ReduceSinkOperator parentRS = (ReduceSinkOperator) parentOp;
+
+            // TableDesc needed for dynamic partitioned hash join
+            GenMapRedUtils.setKeyAndValueDesc(reduceWork, parentRS);
+
+            // For small table RS parents that have already been processed, we need to
+            // add the tag to the RS work to the reduce work that contains this map join.
+            // This was not being done for normal mapjoins, where the small table typically
+            // has its ReduceSink parent removed.
+            if (!context.mapJoinToUnprocessedSmallTableReduceSinks.get(mj).contains(parentRS)) {
+              // This reduce sink has been processed already, so the work for the parentRS exists
+              BaseWork parentWork = ReduceSinkMapJoinProc.getMapJoinParentWork(context, parentRS);
+              int tag = parentRS.getConf().getTag();
+              tag = (tag == -1 ? 0 : tag);
+              reduceWork.getTagToInput().put(tag, parentWork.getName());
+            }
+
+          }
+        }
+
         LOG.debug("Processing map join: " + mj);
         // remember the mapping in case we scan another branch of the
         // mapjoin later
@@ -369,15 +417,44 @@ public class GenTezWork implements NodeProcessor {
         // remember the output name of the reduce sink
         rs.getConf().setOutputName(rWork.getName());
 
+        // For dynamic partitioned hash join, run the ReduceSinkMapJoinProc logic for any
+        // ReduceSink parents that we missed.
+        MapJoinOperator mj = context.smallTableParentToMapJoinMap.get(rs);
+        if (mj != null) {
+          // Only need to run the logic for tables we missed
+          if (context.mapJoinToUnprocessedSmallTableReduceSinks.get(mj).contains(rs)) {
+            // ReduceSinkMapJoinProc logic does not work unless the ReduceSink is connected as
+            // a parent of the MapJoin, but at this point we have already removed all of the
+            // parents from the MapJoin.
+            // Try temporarily adding the RS as a parent
+            ArrayList<Operator<?>> tempMJParents = new ArrayList<Operator<?>>();
+            tempMJParents.add(rs);
+            mj.setParentOperators(tempMJParents);
+            // ReduceSink also needs MapJoin as child
+            List<Operator<?>> rsChildren = rs.getChildOperators();
+            rsChildren.add(mj);
+
+            // Since the MapJoin has had all of its other parents removed at this point,
+            // it would be bad here if processReduceSinkToHashJoin() tries to do anything
+            // with the RS parent based on its position in the list of parents.
+            ReduceSinkMapJoinProc.processReduceSinkToHashJoin(rs, mj, context);
+
+            // Remove any parents from MapJoin again
+            mj.removeParents();
+            // TODO: do we also need to remove the MapJoin from the list of RS's children?
+          }
+        }
+
         if (!context.connectedReduceSinks.contains(rs)) {
           // add dependency between the two work items
           TezEdgeProperty edgeProp;
+          EdgeType edgeType = utils.determineEdgeType(work, followingWork);
           if (rWork.isAutoReduceParallelism()) {
             edgeProp =
-                new TezEdgeProperty(context.conf, EdgeType.SIMPLE_EDGE, true,
+                new TezEdgeProperty(context.conf, edgeType, true,
                     rWork.getMinReduceTasks(), rWork.getMaxReduceTasks(), bytesPerReducer);
           } else {
-            edgeProp = new TezEdgeProperty(EdgeType.SIMPLE_EDGE);
+            edgeProp = new TezEdgeProperty(edgeType);
           }
           tezWork.connect(work, followingWork, edgeProp);
           context.connectedReduceSinks.add(rs);

http://git-wip-us.apache.org/repos/asf/hive/blob/04d54f61/ql/src/java/org/apache/hadoop/hive/ql/plan/BaseWork.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/BaseWork.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/BaseWork.java
index fa697ef..d574c5c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/BaseWork.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/BaseWork.java
@@ -47,7 +47,7 @@ public abstract class BaseWork extends AbstractOperatorDesc {
   // Their function is mainly as root ops to give the mapjoin the correct
   // schema info.
   List<HashTableDummyOperator> dummyOps;
-  int tag;
+  int tag = 0;
   private final List<String> sortColNames = new ArrayList<String>();
 
   private MapredLocalWork mrLocalWork;

http://git-wip-us.apache.org/repos/asf/hive/blob/04d54f61/ql/src/java/org/apache/hadoop/hive/ql/plan/CommonMergeJoinDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/CommonMergeJoinDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/CommonMergeJoinDesc.java
index f9c34cb..cce9bc4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/CommonMergeJoinDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/CommonMergeJoinDesc.java
@@ -45,4 +45,8 @@ public class CommonMergeJoinDesc extends MapJoinDesc implements Serializable {
   public int getBigTablePosition() {
     return mapJoinConversionPos;
   }
+
+  public void setBigTablePosition(int pos) {
+    mapJoinConversionPos = pos;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/04d54f61/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeDescUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeDescUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeDescUtils.java
index fb3c4a3..e291a48 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeDescUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeDescUtils.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluatorFactory;
 import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.UDF;
+import org.apache.hadoop.hive.ql.optimizer.ConstantPropagateProcFactory;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge;
@@ -278,6 +279,59 @@ public class ExprNodeDescUtils {
     throw new SemanticException("Met multiple parent operators");
   }
 
+  public static List<ExprNodeDesc> resolveJoinKeysAsRSColumns(List<ExprNodeDesc> sourceList,
+      Operator<?> reduceSinkOp) {
+    ArrayList<ExprNodeDesc> result = new ArrayList<ExprNodeDesc>(sourceList.size());
+    for (ExprNodeDesc source : sourceList) {
+      ExprNodeDesc newExpr = resolveJoinKeysAsRSColumns(source, reduceSinkOp);
+      if (newExpr == null) {
+        return null;
+      }
+      result.add(newExpr);
+    }
+    return result;
+  }
+
+  /**
+   * Join keys are expressions based on the select operator. Resolve the expressions so they
+   * are based on the ReduceSink operator
+   *   SEL -> RS -> JOIN
+   * @param source
+   * @param reduceSinkOp
+   * @return
+   */
+  public static ExprNodeDesc resolveJoinKeysAsRSColumns(ExprNodeDesc source, Operator<?> reduceSinkOp) {
+    // Assuming this is only being done for join keys. As a result we shouldn't have to recursively
+    // check any nested child expressions, because the result of the expression should exist as an
+    // output column of the ReduceSink operator
+    if (source == null) {
+      return null;
+    }
+
+    // columnExprMap has the reverse of what we need - a mapping of the internal column names
+    // to the ExprNodeDesc from the previous operation.
+    // Find the key/value where the ExprNodeDesc value matches the column we are searching for.
+    // The key portion of the entry will be the internal column name for the join key expression.
+    for (Map.Entry<String, ExprNodeDesc> mapEntry : reduceSinkOp.getColumnExprMap().entrySet()) {
+      if (mapEntry.getValue().isSame(source)) {
+        String columnInternalName = mapEntry.getKey();
+        if (source instanceof ExprNodeColumnDesc) {
+          // The join key is a table column. Create the ExprNodeDesc based on this column.
+          ColumnInfo columnInfo = reduceSinkOp.getSchema().getColumnInfo(columnInternalName);
+          return new ExprNodeColumnDesc(columnInfo);
+        } else {
+          // Join key expression is likely some expression involving functions/operators, so there
+          // is no actual table column for this. But the ReduceSink operator should still have an
+          // output column corresponding to this expression, using the columnInternalName.
+          // TODO: does tableAlias matter for this kind of expression?
+          return new ExprNodeColumnDesc(source.getTypeInfo(), columnInternalName, "", false);
+        }
+      }
+    }
+
+    return null;  // Couldn't find reference to expression
+  }
+
   public static ExprNodeDesc[] extractComparePair(ExprNodeDesc expr1, ExprNodeDesc expr2) {
     expr1 = extractConstant(expr1);
     expr2 = extractConstant(expr2);
@@ -483,4 +537,65 @@ public class ExprNodeDescUtils {
 
     return exprColLst;
   }  
+
+  public static List<ExprNodeDesc> flattenExprList(List<ExprNodeDesc> sourceList) {
+    ArrayList<ExprNodeDesc> result = new ArrayList<ExprNodeDesc>(sourceList.size());
+    for (ExprNodeDesc source : sourceList) {
+      result.add(flattenExpr(source));
+    }
+    return result;
+  }
+
+  /**
+   * A normal reduce operator's rowObjectInspector looks like a struct containing
+   *  nested key/value structs that contain the column values:
+   *  { key: { reducesinkkey0:int }, value: { _col0:int, _col1:int, .. } }
+   *
+   * While the rowObjectInspector looks the same for vectorized queries during
+   * compilation time, within the tasks at query execution the rowObjectInspector
+   * has changed to a flatter structure without nested key/value structs:
+   *  { 'key.reducesinkkey0':int, 'value._col0':int, 'value._col1':int, .. }
+   *
+   * Trying to fetch 'key.reducesinkkey0' by name from the list of flattened
+   * ObjectInspectors does not work because the '.' gets interpreted as a field member,
+   * even though it is a flattened list of column values.
+   * This workaround converts the column name referenced in the ExprNodeDesc
+   * from a nested field name (key.reducesinkkey0) to key_reducesinkkey0,
+   * simply by replacing '.' with '_'.
+   * @param source
+   * @return
+   */
+  public static ExprNodeDesc flattenExpr(ExprNodeDesc source) {
+    if (source instanceof ExprNodeGenericFuncDesc) {
+      // all children expression should be resolved
+      ExprNodeGenericFuncDesc function = (ExprNodeGenericFuncDesc) source.clone();
+      List<ExprNodeDesc> newChildren = flattenExprList(function.getChildren());
+      for (ExprNodeDesc newChild : newChildren) {
+        if (newChild == null) {
+          // Could not resolve all of the function children, fail
+          return null;
+        }
+      }
+      function.setChildren(newChildren);
+      return function;
+    }
+    if (source instanceof ExprNodeColumnDesc) {
+      ExprNodeColumnDesc column = (ExprNodeColumnDesc) source;
+      // Create a new ColumnInfo, replacing STRUCT.COLUMN with STRUCT_COLUMN
+      String newColumn = column.getColumn().replace('.', '_');
+      return new ExprNodeColumnDesc(source.getTypeInfo(), newColumn, column.getTabAlias(), false);
+    }
+    if (source instanceof ExprNodeFieldDesc) {
+      // field expression should be resolved
+      ExprNodeFieldDesc field = (ExprNodeFieldDesc) source.clone();
+      ExprNodeDesc fieldDesc = flattenExpr(field.getDesc());
+      if (fieldDesc == null) {
+        return null;
+      }
+      field.setDesc(fieldDesc);
+      return field;
+    }
+    // constant or null expr, just return
+    return source;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/04d54f61/ql/src/java/org/apache/hadoop/hive/ql/plan/MapJoinDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/MapJoinDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/MapJoinDesc.java
index cee9100..e27b89b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/MapJoinDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/MapJoinDesc.java
@@ -29,6 +29,8 @@ import java.util.Map.Entry;
 import java.util.Set;
 import org.apache.hadoop.hive.ql.plan.Explain.Level;
 
+import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
+
 /**
  * Map Join operator Descriptor implementation.
  *
@@ -71,6 +73,7 @@ public class MapJoinDesc extends JoinDesc implements Serializable {
   protected boolean genJoinKeys = true;
 
   private boolean isHybridHashJoin;
+  private boolean isDynamicPartitionHashJoin = false;
 
   // Extra parameters only for vectorization.
   private VectorMapJoinDesc vectorDesc;
@@ -369,4 +372,12 @@ public class MapJoinDesc extends JoinDesc implements Serializable {
   public boolean getGenJoinKeys() {
     return genJoinKeys;
   }
+
+  public boolean isDynamicPartitionHashJoin() {
+    return isDynamicPartitionHashJoin;
+  }
+
+  public void setDynamicPartitionHashJoin(boolean isDistributedHashJoin) {
+    this.isDynamicPartitionHashJoin = isDistributedHashJoin;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/04d54f61/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceWork.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceWork.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceWork.java
index a78a92e..020d6de 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceWork.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceWork.java
@@ -136,7 +136,7 @@ public class ReduceWork extends BaseWork {
       return null;
     }
     if (valueObjectInspector == null) {
-      valueObjectInspector = getObjectInspector(tagToValueDesc.get(0));
+      valueObjectInspector = getObjectInspector(tagToValueDesc.get(tag));
     }
     return valueObjectInspector;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/04d54f61/ql/src/test/queries/clientpositive/tez_dynpart_hashjoin_1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/tez_dynpart_hashjoin_1.q b/ql/src/test/queries/clientpositive/tez_dynpart_hashjoin_1.q
new file mode 100644
index 0000000..e3325c4
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/tez_dynpart_hashjoin_1.q
@@ -0,0 +1,101 @@
+
+set hive.explain.user=false;
+set hive.auto.convert.join=false;
+set hive.optimize.dynamic.partition.hashjoin=false;
+
+-- First try with regular mergejoin
+explain
+select
+  *
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+order by a.cint;
+
+select
+  *
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+order by a.cint;
+
+explain
+select
+  count(*)
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null;
+
+select
+  count(*)
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null;
+
+explain
+select
+  a.csmallint, count(*) c1
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+group by a.csmallint
+order by c1;
+
+select
+  a.csmallint, count(*) c1
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+group by a.csmallint
+order by c1;
+
+set hive.auto.convert.join=true;
+set hive.optimize.dynamic.partition.hashjoin=true;
+set hive.auto.convert.join.noconditionaltask.size=200000;
+set hive.exec.reducers.bytes.per.reducer=200000;
+
+-- Try with dynamically partitioned hashjoin
+explain
+select
+  *
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+order by a.cint;
+
+select
+  *
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+order by a.cint;
+
+explain
+select
+  count(*)
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null;
+
+select
+  count(*)
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null;
+
+explain
+select
+  a.csmallint, count(*) c1
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+group by a.csmallint
+order by c1;
+
+select
+  a.csmallint, count(*) c1
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+group by a.csmallint
+order by c1;

http://git-wip-us.apache.org/repos/asf/hive/blob/04d54f61/ql/src/test/queries/clientpositive/tez_dynpart_hashjoin_2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/tez_dynpart_hashjoin_2.q b/ql/src/test/queries/clientpositive/tez_dynpart_hashjoin_2.q
new file mode 100644
index 0000000..af4e2b8
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/tez_dynpart_hashjoin_2.q
@@ -0,0 +1,83 @@
+
+set hive.explain.user=false;
+set hive.auto.convert.join=false;
+set hive.optimize.dynamic.partition.hashjoin=false;
+
+-- Multiple tables, and change the order of the big table (alltypesorc)
+-- First try with regular mergejoin
+explain
+select
+  a.*
+from
+  alltypesorc a,
+  src b,
+  src c
+where
+  a.csmallint = cast(b.key as int) and a.csmallint = (cast(c.key as int) + 0)
+  and (a.csmallint < 100)
+order by a.csmallint, a.ctinyint, a.cint;
+
+select
+  a.*
+from
+  alltypesorc a,
+  src b,
+  src c
+where
+  a.csmallint = cast(b.key as int) and a.csmallint = (cast(c.key as int) + 0)
+  and (a.csmallint < 100)
+order by a.csmallint, a.ctinyint, a.cint;
+
+set hive.auto.convert.join=true;
+set hive.optimize.dynamic.partition.hashjoin=true;
+set hive.auto.convert.join.noconditionaltask.size=2000;
+set hive.exec.reducers.bytes.per.reducer=200000;
+
+-- noconditionaltask.size needs to be low enough that entire filtered table results do not fit in one task's hash table
+-- Try with dynamically partitioned hash join 
+explain
+select
+  a.*
+from
+  alltypesorc a,
+  src b,
+  src c
+where
+  a.csmallint = cast(b.key as int) and a.csmallint = (cast(c.key as int) + 0)
+  and (a.csmallint < 100)
+order by a.csmallint, a.ctinyint, a.cint;
+
+select
+  a.*
+from
+  alltypesorc a,
+  src b,
+  src c
+where
+  a.csmallint = cast(b.key as int) and a.csmallint = (cast(c.key as int) + 0)
+  and (a.csmallint < 100)
+order by a.csmallint, a.ctinyint, a.cint;
+
+-- Try different order of tables
+explain
+select
+  a.*
+from
+  src b,
+  alltypesorc a,
+  src c
+where
+  a.csmallint = cast(b.key as int) and a.csmallint = (cast(c.key as int) + 0)
+  and (a.csmallint < 100)
+order by a.csmallint, a.ctinyint, a.cint;
+
+select
+  a.*
+from
+  src b,
+  alltypesorc a,
+  src c
+where
+  a.csmallint = cast(b.key as int) and a.csmallint = (cast(c.key as int) + 0)
+  and (a.csmallint < 100)
+order by a.csmallint, a.ctinyint, a.cint;

http://git-wip-us.apache.org/repos/asf/hive/blob/04d54f61/ql/src/test/queries/clientpositive/tez_vector_dynpart_hashjoin_1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/tez_vector_dynpart_hashjoin_1.q b/ql/src/test/queries/clientpositive/tez_vector_dynpart_hashjoin_1.q
new file mode 100644
index 0000000..65fee16
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/tez_vector_dynpart_hashjoin_1.q
@@ -0,0 +1,102 @@
+
+set hive.explain.user=false;
+set hive.auto.convert.join=false;
+set hive.optimize.dynamic.partition.hashjoin=false;
+
+-- First try with regular mergejoin
+explain
+select
+  *
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+order by a.cint;
+
+select
+  *
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+order by a.cint;
+
+explain
+select
+  count(*)
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null;
+
+select
+  count(*)
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null;
+
+explain
+select
+  a.csmallint, count(*) c1
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+group by a.csmallint
+order by c1;
+
+select
+  a.csmallint, count(*) c1
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+group by a.csmallint
+order by c1;
+
+set hive.auto.convert.join=true;
+set hive.optimize.dynamic.partition.hashjoin=true;
+set hive.auto.convert.join.noconditionaltask.size=200000;
+set hive.exec.reducers.bytes.per.reducer=200000;
+set hive.vectorized.execution.enabled=true;
+
+-- Try with dynamically partitioned hashjoin
+explain
+select
+  *
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+order by a.cint;
+
+select
+  *
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+order by a.cint;
+
+explain
+select
+  count(*)
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null;
+
+select
+  count(*)
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null;
+
+explain
+select
+  a.csmallint, count(*) c1
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+group by a.csmallint
+order by c1;
+
+select
+  a.csmallint, count(*) c1
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+group by a.csmallint
+order by c1;

http://git-wip-us.apache.org/repos/asf/hive/blob/04d54f61/ql/src/test/queries/clientpositive/tez_vector_dynpart_hashjoin_2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/tez_vector_dynpart_hashjoin_2.q b/ql/src/test/queries/clientpositive/tez_vector_dynpart_hashjoin_2.q
new file mode 100644
index 0000000..606f455
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/tez_vector_dynpart_hashjoin_2.q
@@ -0,0 +1,84 @@
+
+set hive.explain.user=false;
+set hive.auto.convert.join=false;
+set hive.optimize.dynamic.partition.hashjoin=false;
+
+-- Multiple tables, and change the order of the big table (alltypesorc)
+-- First try with regular mergejoin
+explain
+select
+  a.*
+from
+  alltypesorc a,
+  src b,
+  src c
+where
+  a.csmallint = cast(b.key as int) and a.csmallint = (cast(c.key as int) + 0)
+  and (a.csmallint < 100)
+order by a.csmallint, a.ctinyint, a.cint;
+
+select
+  a.*
+from
+  alltypesorc a,
+  src b,
+  src c
+where
+  a.csmallint = cast(b.key as int) and a.csmallint = (cast(c.key as int) + 0)
+  and (a.csmallint < 100)
+order by a.csmallint, a.ctinyint, a.cint;
+
+set hive.auto.convert.join=true;
+set hive.optimize.dynamic.partition.hashjoin=true;
+set hive.auto.convert.join.noconditionaltask.size=2000;
+set hive.exec.reducers.bytes.per.reducer=200000;
+set hive.vectorized.execution.enabled=true;
+
+-- noconditionaltask.size needs to be low enough that entire filtered table results do not fit in one task's hash table
+-- Try with dynamically partitioned hash join 
+explain
+select
+  a.*
+from
+  alltypesorc a,
+  src b,
+  src c
+where
+  a.csmallint = cast(b.key as int) and a.csmallint = (cast(c.key as int) + 0)
+  and (a.csmallint < 100)
+order by a.csmallint, a.ctinyint, a.cint;
+
+select
+  a.*
+from
+  alltypesorc a,
+  src b,
+  src c
+where
+  a.csmallint = cast(b.key as int) and a.csmallint = (cast(c.key as int) + 0)
+  and (a.csmallint < 100)
+order by a.csmallint, a.ctinyint, a.cint;
+
+-- Try different order of tables
+explain
+select
+  a.*
+from
+  src b,
+  alltypesorc a,
+  src c
+where
+  a.csmallint = cast(b.key as int) and a.csmallint = (cast(c.key as int) + 0)
+  and (a.csmallint < 100)
+order by a.csmallint, a.ctinyint, a.cint;
+
+select
+  a.*
+from
+  src b,
+  alltypesorc a,
+  src c
+where
+  a.csmallint = cast(b.key as int) and a.csmallint = (cast(c.key as int) + 0)
+  and (a.csmallint < 100)
+order by a.csmallint, a.ctinyint, a.cint;


[28/43] hive git commit: HIVE-10799. Refactor the SearchArgumentFactory to remove the AST-specific factory. (omalley reviewed by prasanth_j)

Posted by xu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/c178a6e9/ql/src/test/org/apache/hadoop/hive/ql/io/sarg/TestConvertAstToSearchArg.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/sarg/TestConvertAstToSearchArg.java b/ql/src/test/org/apache/hadoop/hive/ql/io/sarg/TestConvertAstToSearchArg.java
new file mode 100644
index 0000000..85e952f
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/sarg/TestConvertAstToSearchArg.java
@@ -0,0 +1,2863 @@
+/**
+ * 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.io.sarg;
+
+import static junit.framework.Assert.assertEquals;
+import static junit.framework.Assert.assertNull;
+import static junit.framework.Assert.assertTrue;
+
+import com.google.common.collect.Sets;
+
+import org.apache.hadoop.hive.common.type.HiveChar;
+import org.apache.hadoop.hive.common.type.HiveVarchar;
+import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument.TruthValue;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentImpl.PredicateLeafImpl;
+import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.junit.Test;
+
+import java.beans.XMLDecoder;
+import java.io.ByteArrayInputStream;
+import java.io.UnsupportedEncodingException;
+import java.lang.reflect.Field;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+
+/**
+ * These tests cover the conversion from Hive's AST to SearchArguments.
+ */
+public class TestConvertAstToSearchArg {
+
+  private static void assertNoSharedNodes(ExpressionTree tree,
+                                          Set<ExpressionTree> seen
+                                          ) throws Exception {
+    if (seen.contains(tree) &&
+        tree.getOperator() != ExpressionTree.Operator.LEAF) {
+      assertTrue("repeated node in expression " + tree, false);
+    }
+    seen.add(tree);
+    if (tree.getChildren() != null) {
+      for (ExpressionTree child : tree.getChildren()) {
+        assertNoSharedNodes(child, seen);
+      }
+    }
+  }
+
+  private ExprNodeGenericFuncDesc getFuncDesc(String xmlSerialized) {
+    byte[] bytes;
+    try {
+      bytes = xmlSerialized.getBytes("UTF-8");
+    } catch (UnsupportedEncodingException ex) {
+      throw new RuntimeException("UTF-8 support required", ex);
+    }
+
+    ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
+    XMLDecoder decoder = new XMLDecoder(bais, null, null);
+
+    try {
+      return (ExprNodeGenericFuncDesc) decoder.readObject();
+    } finally {
+      decoder.close();
+    }
+  }
+
+  @Test
+  public void testExpression1() throws Exception {
+    // first_name = 'john' or
+    //  'greg' < first_name or
+    //  'alan' > first_name or
+    //  id > 12 or
+    //  13 < id or
+    //  id < 15 or
+    //  16 > id or
+    //  (id <=> 30 and first_name <=> 'owen')
+    String exprStr = "<?xml version=\"1.0\" encoding=\"UTF-8\"?> \n" +
+        "<java version=\"1.6.0_31\" class=\"java.beans.XMLDecoder\"> \n" +
+        " <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "  <void property=\"children\"> \n" +
+        "   <object class=\"java.util.ArrayList\"> \n" +
+        "    <void method=\"add\"> \n" +
+        "     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "      <void property=\"children\"> \n" +
+        "       <object class=\"java.util.ArrayList\"> \n" +
+        "        <void method=\"add\"> \n" +
+        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "          <void property=\"children\"> \n" +
+        "           <object class=\"java.util.ArrayList\"> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "              <void property=\"children\"> \n" +
+        "               <object class=\"java.util.ArrayList\"> \n" +
+        "                <void method=\"add\"> \n" +
+        "                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "                  <void property=\"children\"> \n" +
+        "                   <object class=\"java.util.ArrayList\"> \n" +
+        "                    <void method=\"add\"> \n" +
+        "                     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "                      <void property=\"children\"> \n" +
+        "                       <object class=\"java.util.ArrayList\"> \n" +
+        "                        <void method=\"add\"> \n" +
+        "                         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "                          <void property=\"children\"> \n" +
+        "                           <object class=\"java.util.ArrayList\"> \n" +
+        "                            <void method=\"add\"> \n" +
+        "                             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "                              <void property=\"children\"> \n" +
+        "                               <object class=\"java.util.ArrayList\"> \n" +
+        "                                <void method=\"add\"> \n" +
+        "                                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "                                  <void property=\"column\"> \n" +
+        "                                   <string>first_name</string> \n" +
+        "                                  </void> \n" +
+        "                                  <void property=\"tabAlias\"> \n" +
+        "                                   <string>orc_people</string> \n" +
+        "                                  </void> \n" +
+        "                                  <void property=\"typeInfo\"> \n" +
+        "                                   <object id=\"PrimitiveTypeInfo0\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
+        "                                    <void property=\"typeName\"> \n" +
+        "                                     <string>string</string> \n" +
+        "                                    </void> \n" +
+        "                                   </object> \n" +
+        "                                  </void> \n" +
+        "                                 </object> \n" +
+        "                                </void> \n" +
+        "                                <void method=\"add\"> \n" +
+        "                                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "                                  <void property=\"typeInfo\"> \n" +
+        "                                   <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "                                  </void> \n" +
+        "                                  <void property=\"value\"> \n" +
+        "                                   <string>john</string> \n" +
+        "                                  </void> \n" +
+        "                                 </object> \n" +
+        "                                </void> \n" +
+        "                               </object> \n" +
+        "                              </void> \n" +
+        "                              <void property=\"genericUDF\"> \n" +
+        "                               <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual\"/> \n" +
+        "                              </void> \n" +
+        "                              <void property=\"typeInfo\"> \n" +
+        "                               <object id=\"PrimitiveTypeInfo1\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
+        "                                <void property=\"typeName\"> \n" +
+        "                                 <string>boolean</string> \n" +
+        "                                </void> \n" +
+        "                               </object> \n" +
+        "                              </void> \n" +
+        "                             </object> \n" +
+        "                            </void> \n" +
+        "                            <void method=\"add\"> \n" +
+        "                             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "                              <void property=\"children\"> \n" +
+        "                               <object class=\"java.util.ArrayList\"> \n" +
+        "                                <void method=\"add\"> \n" +
+        "                                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "                                  <void property=\"typeInfo\"> \n" +
+        "                                   <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "                                  </void> \n" +
+        "                                  <void property=\"value\"> \n" +
+        "                                   <string>greg</string> \n" +
+        "                                  </void> \n" +
+        "                                 </object> \n" +
+        "                                </void> \n" +
+        "                                <void method=\"add\"> \n" +
+        "                                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "                                  <void property=\"column\"> \n" +
+        "                                   <string>first_name</string> \n" +
+        "                                  </void> \n" +
+        "                                  <void property=\"tabAlias\"> \n" +
+        "                                   <string>orc_people</string> \n" +
+        "                                  </void> \n" +
+        "                                  <void property=\"typeInfo\"> \n" +
+        "                                   <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "                                  </void> \n" +
+        "                                 </object> \n" +
+        "                                </void> \n" +
+        "                               </object> \n" +
+        "                              </void> \n" +
+        "                              <void property=\"genericUDF\"> \n" +
+        "                               <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPLessThan\"/> \n" +
+        "                              </void> \n" +
+        "                              <void property=\"typeInfo\"> \n" +
+        "                               <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "                              </void> \n" +
+        "                             </object> \n" +
+        "                            </void> \n" +
+        "                           </object> \n" +
+        "                          </void> \n" +
+        "                          <void property=\"genericUDF\"> \n" +
+        "                           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr\"/> \n" +
+        "                          </void> \n" +
+        "                          <void property=\"typeInfo\"> \n" +
+        "                           <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "                          </void> \n" +
+        "                         </object> \n" +
+        "                        </void> \n" +
+        "                        <void method=\"add\"> \n" +
+        "                         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "                          <void property=\"children\"> \n" +
+        "                           <object class=\"java.util.ArrayList\"> \n" +
+        "                            <void method=\"add\"> \n" +
+        "                             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "                              <void property=\"typeInfo\"> \n" +
+        "                               <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "                              </void> \n" +
+        "                              <void property=\"value\"> \n" +
+        "                               <string>alan</string> \n" +
+        "                              </void> \n" +
+        "                             </object> \n" +
+        "                            </void> \n" +
+        "                            <void method=\"add\"> \n" +
+        "                             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "                              <void property=\"column\"> \n" +
+        "                               <string>first_name</string> \n" +
+        "                              </void> \n" +
+        "                              <void property=\"tabAlias\"> \n" +
+        "                               <string>orc_people</string> \n" +
+        "                              </void> \n" +
+        "                              <void property=\"typeInfo\"> \n" +
+        "                               <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "                              </void> \n" +
+        "                             </object> \n" +
+        "                            </void> \n" +
+        "                           </object> \n" +
+        "                          </void> \n" +
+        "                          <void property=\"genericUDF\"> \n" +
+        "                           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPGreaterThan\"/> \n" +
+        "                          </void> \n" +
+        "                          <void property=\"typeInfo\"> \n" +
+        "                           <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "                          </void> \n" +
+        "                         </object> \n" +
+        "                        </void> \n" +
+        "                       </object> \n" +
+        "                      </void> \n" +
+        "                      <void property=\"genericUDF\"> \n" +
+        "                       <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr\"/> \n" +
+        "                      </void> \n" +
+        "                      <void property=\"typeInfo\"> \n" +
+        "                       <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "                      </void> \n" +
+        "                     </object> \n" +
+        "                    </void> \n" +
+        "                    <void method=\"add\"> \n" +
+        "                     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "                      <void property=\"children\"> \n" +
+        "                       <object class=\"java.util.ArrayList\"> \n" +
+        "                        <void method=\"add\"> \n" +
+        "                         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "                          <void property=\"column\"> \n" +
+        "                           <string>id</string> \n" +
+        "                          </void> \n" +
+        "                          <void property=\"tabAlias\"> \n" +
+        "                           <string>orc_people</string> \n" +
+        "                          </void> \n" +
+        "                          <void property=\"typeInfo\"> \n" +
+        "                           <object id=\"PrimitiveTypeInfo2\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
+        "                            <void property=\"typeName\"> \n" +
+        "                             <string>int</string> \n" +
+        "                            </void> \n" +
+        "                           </object> \n" +
+        "                          </void> \n" +
+        "                         </object> \n" +
+        "                        </void> \n" +
+        "                        <void method=\"add\"> \n" +
+        "                         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "                          <void property=\"typeInfo\"> \n" +
+        "                           <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "                          </void> \n" +
+        "                          <void property=\"value\"> \n" +
+        "                           <int>12</int> \n" +
+        "                          </void> \n" +
+        "                         </object> \n" +
+        "                        </void> \n" +
+        "                       </object> \n" +
+        "                      </void> \n" +
+        "                      <void property=\"genericUDF\"> \n" +
+        "                       <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPGreaterThan\"/> \n" +
+        "                      </void> \n" +
+        "                      <void property=\"typeInfo\"> \n" +
+        "                       <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "                      </void> \n" +
+        "                     </object> \n" +
+        "                    </void> \n" +
+        "                   </object> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"genericUDF\"> \n" +
+        "                   <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr\"/> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"typeInfo\"> \n" +
+        "                   <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "                  </void> \n" +
+        "                 </object> \n" +
+        "                </void> \n" +
+        "                <void method=\"add\"> \n" +
+        "                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "                  <void property=\"children\"> \n" +
+        "                   <object class=\"java.util.ArrayList\"> \n" +
+        "                    <void method=\"add\"> \n" +
+        "                     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "                      <void property=\"typeInfo\"> \n" +
+        "                       <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "                      </void> \n" +
+        "                      <void property=\"value\"> \n" +
+        "                       <int>13</int> \n" +
+        "                      </void> \n" +
+        "                     </object> \n" +
+        "                    </void> \n" +
+        "                    <void method=\"add\"> \n" +
+        "                     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "                      <void property=\"column\"> \n" +
+        "                       <string>id</string> \n" +
+        "                      </void> \n" +
+        "                      <void property=\"tabAlias\"> \n" +
+        "                       <string>orc_people</string> \n" +
+        "                      </void> \n" +
+        "                      <void property=\"typeInfo\"> \n" +
+        "                       <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "                      </void> \n" +
+        "                     </object> \n" +
+        "                    </void> \n" +
+        "                   </object> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"genericUDF\"> \n" +
+        "                   <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPLessThan\"/> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"typeInfo\"> \n" +
+        "                   <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "                  </void> \n" +
+        "                 </object> \n" +
+        "                </void> \n" +
+        "               </object> \n" +
+        "              </void> \n" +
+        "              <void property=\"genericUDF\"> \n" +
+        "               <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr\"/> \n" +
+        "              </void> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "              <void property=\"children\"> \n" +
+        "               <object class=\"java.util.ArrayList\"> \n" +
+        "                <void method=\"add\"> \n" +
+        "                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "                  <void property=\"column\"> \n" +
+        "                   <string>id</string> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"tabAlias\"> \n" +
+        "                   <string>orc_people</string> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"typeInfo\"> \n" +
+        "                   <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "                  </void> \n" +
+        "                 </object> \n" +
+        "                </void> \n" +
+        "                <void method=\"add\"> \n" +
+        "                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "                  <void property=\"typeInfo\"> \n" +
+        "                   <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"value\"> \n" +
+        "                   <int>15</int> \n" +
+        "                  </void> \n" +
+        "                 </object> \n" +
+        "                </void> \n" +
+        "               </object> \n" +
+        "              </void> \n" +
+        "              <void property=\"genericUDF\"> \n" +
+        "               <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPLessThan\"/> \n" +
+        "              </void> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "           </object> \n" +
+        "          </void> \n" +
+        "          <void property=\"genericUDF\"> \n" +
+        "           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr\"/> \n" +
+        "          </void> \n" +
+        "          <void property=\"typeInfo\"> \n" +
+        "           <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "          </void> \n" +
+        "         </object> \n" +
+        "        </void> \n" +
+        "        <void method=\"add\"> \n" +
+        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "          <void property=\"children\"> \n" +
+        "           <object class=\"java.util.ArrayList\"> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "              </void> \n" +
+        "              <void property=\"value\"> \n" +
+        "               <int>16</int> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "              <void property=\"column\"> \n" +
+        "               <string>id</string> \n" +
+        "              </void> \n" +
+        "              <void property=\"tabAlias\"> \n" +
+        "               <string>orc_people</string> \n" +
+        "              </void> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "           </object> \n" +
+        "          </void> \n" +
+        "          <void property=\"genericUDF\"> \n" +
+        "           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPGreaterThan\"/> \n" +
+        "          </void> \n" +
+        "          <void property=\"typeInfo\"> \n" +
+        "           <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "          </void> \n" +
+        "         </object> \n" +
+        "        </void> \n" +
+        "       </object> \n" +
+        "      </void> \n" +
+        "      <void property=\"genericUDF\"> \n" +
+        "       <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr\"/> \n" +
+        "      </void> \n" +
+        "      <void property=\"typeInfo\"> \n" +
+        "       <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "      </void> \n" +
+        "     </object> \n" +
+        "    </void> \n" +
+        "    <void method=\"add\"> \n" +
+        "     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "      <void property=\"children\"> \n" +
+        "       <object class=\"java.util.ArrayList\"> \n" +
+        "        <void method=\"add\"> \n" +
+        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "          <void property=\"children\"> \n" +
+        "           <object class=\"java.util.ArrayList\"> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "              <void property=\"column\"> \n" +
+        "               <string>id</string> \n" +
+        "              </void> \n" +
+        "              <void property=\"tabAlias\"> \n" +
+        "               <string>orc_people</string> \n" +
+        "              </void> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "              </void> \n" +
+        "              <void property=\"value\"> \n" +
+        "               <int>30</int> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "           </object> \n" +
+        "          </void> \n" +
+        "          <void property=\"genericUDF\"> \n" +
+        "           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualNS\"/> \n" +
+        "          </void> \n" +
+        "          <void property=\"typeInfo\"> \n" +
+        "           <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "          </void> \n" +
+        "         </object> \n" +
+        "        </void> \n" +
+        "        <void method=\"add\"> \n" +
+        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "          <void property=\"children\"> \n" +
+        "           <object class=\"java.util.ArrayList\"> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "              <void property=\"column\"> \n" +
+        "               <string>first_name</string> \n" +
+        "              </void> \n" +
+        "              <void property=\"tabAlias\"> \n" +
+        "               <string>orc_people</string> \n" +
+        "              </void> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "              </void> \n" +
+        "              <void property=\"value\"> \n" +
+        "               <string>owen</string> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "           </object> \n" +
+        "          </void> \n" +
+        "          <void property=\"genericUDF\"> \n" +
+        "           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualNS\"/> \n" +
+        "          </void> \n" +
+        "          <void property=\"typeInfo\"> \n" +
+        "           <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "          </void> \n" +
+        "         </object> \n" +
+        "        </void> \n" +
+        "       </object> \n" +
+        "      </void> \n" +
+        "      <void property=\"genericUDF\"> \n" +
+        "       <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPAnd\"/> \n" +
+        "      </void> \n" +
+        "      <void property=\"typeInfo\"> \n" +
+        "       <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "      </void> \n" +
+        "     </object> \n" +
+        "    </void> \n" +
+        "   </object> \n" +
+        "  </void> \n" +
+        "  <void property=\"genericUDF\"> \n" +
+        "   <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr\"/> \n" +
+        "  </void> \n" +
+        "  <void property=\"typeInfo\"> \n" +
+        "   <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "  </void> \n" +
+        " </object> \n" +
+        "</java> \n";
+
+    SearchArgumentImpl sarg =
+        (SearchArgumentImpl) ConvertAstToSearchArg.create(getFuncDesc(exprStr));
+    List<PredicateLeaf> leaves = sarg.getLeaves();
+    assertEquals(9, leaves.size());
+
+    FilterPredicate p = ParquetRecordReaderWrapper.toFilterPredicate(sarg);
+    String[] conditions = new String[]{
+      "eq(first_name, Binary{\"john\"})",    /* first_name = 'john' */
+      "not(lteq(first_name, Binary{\"greg\"}))", /* 'greg' < first_name */
+      "lt(first_name, Binary{\"alan\"})",   /* 'alan' > first_name */
+      "not(lteq(id, 12))",                  /* id > 12 or */
+      "not(lteq(id, 13))",                  /* 13 < id or */
+      "lt(id, 15)",                         /* id < 15 or */
+      "lt(id, 16)",                         /* 16 > id or */
+      "eq(id, 30)",                         /* id <=> 30 */
+      "eq(first_name, Binary{\"owen\"})"    /* first_name <=> 'owen' */
+    };
+    String expected = String
+      .format("and(or(or(or(or(or(or(or(%1$s, %2$s), %3$s), %4$s), %5$s), %6$s), %7$s), %8$s), " +
+        "or(or(or(or(or(or(or(%1$s, %2$s), %3$s), %4$s), %5$s), %6$s), %7$s), %9$s))", conditions);
+    assertEquals(expected, p.toString());
+
+    PredicateLeaf leaf = leaves.get(0);
+    assertEquals(PredicateLeaf.Type.STRING, leaf.getType());
+    assertEquals(PredicateLeaf.Operator.EQUALS, leaf.getOperator());
+    assertEquals("first_name", leaf.getColumnName());
+    assertEquals("john", leaf.getLiteral());
+
+    leaf = leaves.get(1);
+    assertEquals(PredicateLeaf.Type.STRING, leaf.getType());
+    assertEquals(PredicateLeaf.Operator.LESS_THAN_EQUALS, leaf.getOperator());
+    assertEquals("first_name", leaf.getColumnName());
+    assertEquals("greg", leaf.getLiteral());
+
+    leaf = leaves.get(2);
+    assertEquals(PredicateLeaf.Type.STRING, leaf.getType());
+    assertEquals(PredicateLeaf.Operator.LESS_THAN, leaf.getOperator());
+    assertEquals("first_name", leaf.getColumnName());
+    assertEquals("alan", leaf.getLiteral());
+
+    leaf = leaves.get(3);
+    assertEquals(PredicateLeaf.Type.INTEGER, leaf.getType());
+    assertEquals(PredicateLeaf.Operator.LESS_THAN_EQUALS, leaf.getOperator());
+    assertEquals("id", leaf.getColumnName());
+    assertEquals(12, leaf.getLiteral());
+
+    leaf = leaves.get(4);
+    assertEquals(PredicateLeaf.Type.INTEGER, leaf.getType());
+    assertEquals(PredicateLeaf.Operator.LESS_THAN_EQUALS, leaf.getOperator());
+    assertEquals("id", leaf.getColumnName());
+    assertEquals(13, leaf.getLiteral());
+
+    leaf = leaves.get(5);
+    assertEquals(PredicateLeaf.Type.INTEGER, leaf.getType());
+    assertEquals(PredicateLeaf.Operator.LESS_THAN, leaf.getOperator());
+    assertEquals("id", leaf.getColumnName());
+    assertEquals(15, leaf.getLiteral());
+
+    leaf = leaves.get(6);
+    assertEquals(PredicateLeaf.Type.INTEGER, leaf.getType());
+    assertEquals(PredicateLeaf.Operator.LESS_THAN, leaf.getOperator());
+    assertEquals("id", leaf.getColumnName());
+    assertEquals(16, leaf.getLiteral());
+
+    leaf = leaves.get(7);
+    assertEquals(PredicateLeaf.Type.INTEGER, leaf.getType());
+    assertEquals(PredicateLeaf.Operator.NULL_SAFE_EQUALS, leaf.getOperator());
+    assertEquals("id", leaf.getColumnName());
+    assertEquals(30, leaf.getLiteral());
+
+    leaf = leaves.get(8);
+    assertEquals(PredicateLeaf.Type.STRING, leaf.getType());
+    assertEquals(PredicateLeaf.Operator.NULL_SAFE_EQUALS, leaf.getOperator());
+    assertEquals("first_name", leaf.getColumnName());
+    assertEquals("owen", leaf.getLiteral());
+
+    assertEquals("(and (or leaf-0 (not leaf-1) leaf-2 (not leaf-3)" +
+        " (not leaf-4) leaf-5 leaf-6 leaf-7)" +
+        " (or leaf-0 (not leaf-1) leaf-2 (not leaf-3)" +
+        " (not leaf-4) leaf-5 leaf-6 leaf-8))",
+        sarg.getExpression().toString());
+    assertNoSharedNodes(sarg.getExpression(),
+        Sets.<ExpressionTree>newIdentityHashSet());
+  }
+
+  @Test
+  public void testExpression2() throws Exception {
+    /* first_name is null or
+       first_name <> 'sue' or
+       id >= 12 or
+       id <= 4; */
+    String exprStr = "<?xml version=\"1.0\" encoding=\"UTF-8\"?> \n" +
+        "<java version=\"1.6.0_31\" class=\"java.beans.XMLDecoder\"> \n" +
+        " <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "  <void property=\"children\"> \n" +
+        "   <object class=\"java.util.ArrayList\"> \n" +
+        "    <void method=\"add\"> \n" +
+        "     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "      <void property=\"children\"> \n" +
+        "       <object class=\"java.util.ArrayList\"> \n" +
+        "        <void method=\"add\"> \n" +
+        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "          <void property=\"children\"> \n" +
+        "           <object class=\"java.util.ArrayList\"> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "              <void property=\"children\"> \n" +
+        "               <object class=\"java.util.ArrayList\"> \n" +
+        "                <void method=\"add\"> \n" +
+        "                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "                  <void property=\"column\"> \n" +
+        "                   <string>first_name</string> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"tabAlias\"> \n" +
+        "                   <string>orc_people</string> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"typeInfo\"> \n" +
+        "                   <object id=\"PrimitiveTypeInfo0\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
+        "                    <void property=\"typeName\"> \n" +
+        "                     <string>string</string> \n" +
+        "                    </void> \n" +
+        "                   </object> \n" +
+        "                  </void> \n" +
+        "                 </object> \n" +
+        "                </void> \n" +
+        "               </object> \n" +
+        "              </void> \n" +
+        "              <void property=\"genericUDF\"> \n" +
+        "               <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNull\"/> \n" +
+        "              </void> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object id=\"PrimitiveTypeInfo1\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
+        "                <void property=\"typeName\"> \n" +
+        "                 <string>boolean</string> \n" +
+        "                </void> \n" +
+        "               </object> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "              <void property=\"children\"> \n" +
+        "               <object class=\"java.util.ArrayList\"> \n" +
+        "                <void method=\"add\"> \n" +
+        "                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "                  <void property=\"column\"> \n" +
+        "                   <string>first_name</string> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"tabAlias\"> \n" +
+        "                   <string>orc_people</string> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"typeInfo\"> \n" +
+        "                   <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "                  </void> \n" +
+        "                 </object> \n" +
+        "                </void> \n" +
+        "                <void method=\"add\"> \n" +
+        "                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "                  <void property=\"typeInfo\"> \n" +
+        "                   <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"value\"> \n" +
+        "                   <string>sue</string> \n" +
+        "                  </void> \n" +
+        "                 </object> \n" +
+        "                </void> \n" +
+        "               </object> \n" +
+        "              </void> \n" +
+        "              <void property=\"genericUDF\"> \n" +
+        "               <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNotEqual\"/> \n" +
+        "              </void> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "           </object> \n" +
+        "          </void> \n" +
+        "          <void property=\"genericUDF\"> \n" +
+        "           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr\"/> \n" +
+        "          </void> \n" +
+        "          <void property=\"typeInfo\"> \n" +
+        "           <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "          </void> \n" +
+        "         </object> \n" +
+        "        </void> \n" +
+        "        <void method=\"add\"> \n" +
+        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "          <void property=\"children\"> \n" +
+        "           <object class=\"java.util.ArrayList\"> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "              <void property=\"column\"> \n" +
+        "               <string>id</string> \n" +
+        "              </void> \n" +
+        "              <void property=\"tabAlias\"> \n" +
+        "               <string>orc_people</string> \n" +
+        "              </void> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object id=\"PrimitiveTypeInfo2\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
+        "                <void property=\"typeName\"> \n" +
+        "                 <string>int</string> \n" +
+        "                </void> \n" +
+        "               </object> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "              </void> \n" +
+        "              <void property=\"value\"> \n" +
+        "               <int>12</int> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "           </object> \n" +
+        "          </void> \n" +
+        "          <void property=\"genericUDF\"> \n" +
+        "           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrGreaterThan\"/> \n" +
+        "          </void> \n" +
+        "          <void property=\"typeInfo\"> \n" +
+        "           <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "          </void> \n" +
+        "         </object> \n" +
+        "        </void> \n" +
+        "       </object> \n" +
+        "      </void> \n" +
+        "      <void property=\"genericUDF\"> \n" +
+        "       <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr\"/> \n" +
+        "      </void> \n" +
+        "      <void property=\"typeInfo\"> \n" +
+        "       <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "      </void> \n" +
+        "     </object> \n" +
+        "    </void> \n" +
+        "    <void method=\"add\"> \n" +
+        "     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "      <void property=\"children\"> \n" +
+        "       <object class=\"java.util.ArrayList\"> \n" +
+        "        <void method=\"add\"> \n" +
+        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "          <void property=\"column\"> \n" +
+        "           <string>id</string> \n" +
+        "          </void> \n" +
+        "          <void property=\"tabAlias\"> \n" +
+        "           <string>orc_people</string> \n" +
+        "          </void> \n" +
+        "          <void property=\"typeInfo\"> \n" +
+        "           <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "          </void> \n" +
+        "         </object> \n" +
+        "        </void> \n" +
+        "        <void method=\"add\"> \n" +
+        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "          <void property=\"typeInfo\"> \n" +
+        "           <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "          </void> \n" +
+        "          <void property=\"value\"> \n" +
+        "           <int>4</int> \n" +
+        "          </void> \n" +
+        "         </object> \n" +
+        "        </void> \n" +
+        "       </object> \n" +
+        "      </void> \n" +
+        "      <void property=\"genericUDF\"> \n" +
+        "       <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrLessThan\"/> \n" +
+        "      </void> \n" +
+        "      <void property=\"typeInfo\"> \n" +
+        "       <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "      </void> \n" +
+        "     </object> \n" +
+        "    </void> \n" +
+        "   </object> \n" +
+        "  </void> \n" +
+        "  <void property=\"genericUDF\"> \n" +
+        "   <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr\"/> \n" +
+        "  </void> \n" +
+        "  <void property=\"typeInfo\"> \n" +
+        "   <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "  </void> \n" +
+        " </object> \n" +
+        "</java> \n";
+
+    SearchArgumentImpl sarg =
+        (SearchArgumentImpl) ConvertAstToSearchArg.create(getFuncDesc(exprStr));
+    List<PredicateLeaf> leaves = sarg.getLeaves();
+    assertEquals(4, leaves.size());
+
+    String[] conditions = new String[]{
+      "eq(first_name, null)",               /* first_name is null  */
+      "not(eq(first_name, Binary{\"sue\"}))",    /* first_name <> 'sue' */
+      "not(lt(id, 12))",                    /* id >= 12            */
+      "lteq(id, 4)"                         /* id <= 4             */
+    };
+
+    FilterPredicate p = ParquetRecordReaderWrapper.toFilterPredicate(sarg);
+    String expected = String.format("or(or(or(%1$s, %2$s), %3$s), %4$s)", conditions);
+    assertEquals(expected, p.toString());
+
+    PredicateLeaf leaf = leaves.get(0);
+    assertEquals(PredicateLeaf.Type.STRING, leaf.getType());
+    assertEquals(PredicateLeaf.Operator.IS_NULL, leaf.getOperator());
+    assertEquals("first_name", leaf.getColumnName());
+    assertEquals(null, leaf.getLiteral());
+    assertEquals(null, leaf.getLiteralList());
+
+    leaf = leaves.get(1);
+    assertEquals(PredicateLeaf.Type.STRING, leaf.getType());
+    assertEquals(PredicateLeaf.Operator.EQUALS, leaf.getOperator());
+    assertEquals("first_name", leaf.getColumnName());
+    assertEquals("sue", leaf.getLiteral());
+
+    leaf = leaves.get(2);
+    assertEquals(PredicateLeaf.Type.INTEGER, leaf.getType());
+    assertEquals(PredicateLeaf.Operator.LESS_THAN, leaf.getOperator());
+    assertEquals("id", leaf.getColumnName());
+    assertEquals(12, leaf.getLiteral());
+
+    leaf = leaves.get(3);
+    assertEquals(PredicateLeaf.Type.INTEGER, leaf.getType());
+    assertEquals(PredicateLeaf.Operator.LESS_THAN_EQUALS, leaf.getOperator());
+    assertEquals("id", leaf.getColumnName());
+    assertEquals(4, leaf.getLiteral());
+
+    assertEquals("(or leaf-0 (not leaf-1) (not leaf-2) leaf-3)",
+        sarg.getExpression().toString());
+    assertNoSharedNodes(sarg.getExpression(),
+        Sets.<ExpressionTree>newIdentityHashSet());
+    assertEquals(TruthValue.NO,
+        sarg.evaluate(values(TruthValue.NO, TruthValue.YES, TruthValue.YES,
+            TruthValue.NO)));
+    assertEquals(TruthValue.YES,
+        sarg.evaluate(values(TruthValue.YES, TruthValue.YES, TruthValue.YES,
+            TruthValue.NO)));
+    assertEquals(TruthValue.YES,
+        sarg.evaluate(values(TruthValue.NO, TruthValue.NO, TruthValue.YES,
+            TruthValue.NO)));
+    assertEquals(TruthValue.YES,
+        sarg.evaluate(values(TruthValue.NO, TruthValue.YES, TruthValue.NO,
+            TruthValue.NO)));
+    assertEquals(TruthValue.YES,
+        sarg.evaluate(values(TruthValue.NO, TruthValue.YES, TruthValue.YES,
+            TruthValue.YES)));
+    assertEquals(TruthValue.NULL,
+        sarg.evaluate(values(TruthValue.NULL, TruthValue.YES, TruthValue.YES,
+            TruthValue.NO)));
+    assertEquals(TruthValue.NULL,
+        sarg.evaluate(values(TruthValue.NO, TruthValue.NULL, TruthValue.YES,
+            TruthValue.NO)));
+    assertEquals(TruthValue.NULL,
+        sarg.evaluate(values(TruthValue.NO, TruthValue.YES, TruthValue.NULL,
+            TruthValue.NO)));
+    assertEquals(TruthValue.NULL,
+        sarg.evaluate(values(TruthValue.NO, TruthValue.YES, TruthValue.YES,
+            TruthValue.NULL)));
+    assertEquals(TruthValue.YES_NO,
+        sarg.evaluate(values(TruthValue.NO, TruthValue.YES_NO, TruthValue.YES,
+            TruthValue.YES_NO)));
+    assertEquals(TruthValue.NO_NULL,
+        sarg.evaluate(values(TruthValue.NO, TruthValue.YES_NULL, TruthValue.YES,
+            TruthValue.NO_NULL)));
+    assertEquals(TruthValue.YES_NULL,
+        sarg.evaluate(values(TruthValue.YES_NULL, TruthValue.YES_NO_NULL,
+            TruthValue.YES, TruthValue.NULL)));
+    assertEquals(TruthValue.YES_NO_NULL,
+        sarg.evaluate(values(TruthValue.NO_NULL, TruthValue.YES_NO_NULL,
+            TruthValue.YES, TruthValue.NO)));
+  }
+
+  @Test
+  public void testExpression3() throws Exception {
+    /* (id between 23 and 45) and
+       first_name = 'alan' and
+       substr('xxxxx', 3) == first_name and
+       'smith' = last_name and
+       substr(first_name, 3) == 'yyy' */
+    String exprStr = "<?xml version=\"1.0\" encoding=\"UTF-8\"?> \n" +
+        "<java version=\"1.6.0_31\" class=\"java.beans.XMLDecoder\"> \n" +
+        " <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "  <void property=\"children\"> \n" +
+        "   <object class=\"java.util.ArrayList\"> \n" +
+        "    <void method=\"add\"> \n" +
+        "     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "      <void property=\"children\"> \n" +
+        "       <object class=\"java.util.ArrayList\"> \n" +
+        "        <void method=\"add\"> \n" +
+        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "          <void property=\"children\"> \n" +
+        "           <object class=\"java.util.ArrayList\"> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "              <void property=\"children\"> \n" +
+        "               <object class=\"java.util.ArrayList\"> \n" +
+        "                <void method=\"add\"> \n" +
+        "                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "                  <void property=\"children\"> \n" +
+        "                   <object class=\"java.util.ArrayList\"> \n" +
+        "                    <void method=\"add\"> \n" +
+        "                     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "                      <void property=\"typeInfo\"> \n" +
+        "                       <object id=\"PrimitiveTypeInfo0\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
+        "                        <void property=\"typeName\"> \n" +
+        "                         <string>boolean</string> \n" +
+        "                        </void> \n" +
+        "                       </object> \n" +
+        "                      </void> \n" +
+        "                      <void property=\"value\"> \n" +
+        "                       <boolean>false</boolean> \n" +
+        "                      </void> \n" +
+        "                     </object> \n" +
+        "                    </void> \n" +
+        "                    <void method=\"add\"> \n" +
+        "                     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "                      <void property=\"column\"> \n" +
+        "                       <string>id</string> \n" +
+        "                      </void> \n" +
+        "                      <void property=\"tabAlias\"> \n" +
+        "                       <string>orc_people</string> \n" +
+        "                      </void> \n" +
+        "                      <void property=\"typeInfo\"> \n" +
+        "                       <object id=\"PrimitiveTypeInfo1\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
+        "                        <void property=\"typeName\"> \n" +
+        "                         <string>int</string> \n" +
+        "                        </void> \n" +
+        "                       </object> \n" +
+        "                      </void> \n" +
+        "                     </object> \n" +
+        "                    </void> \n" +
+        "                    <void method=\"add\"> \n" +
+        "                     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "                      <void property=\"typeInfo\"> \n" +
+        "                       <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "                      </void> \n" +
+        "                      <void property=\"value\"> \n" +
+        "                       <int>23</int> \n" +
+        "                      </void> \n" +
+        "                     </object> \n" +
+        "                    </void> \n" +
+        "                    <void method=\"add\"> \n" +
+        "                     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "                      <void property=\"typeInfo\"> \n" +
+        "                       <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "                      </void> \n" +
+        "                      <void property=\"value\"> \n" +
+        "                       <int>45</int> \n" +
+        "                      </void> \n" +
+        "                     </object> \n" +
+        "                    </void> \n" +
+        "                   </object> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"genericUDF\"> \n" +
+        "                   <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFBetween\"/> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"typeInfo\"> \n" +
+        "                   <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "                  </void> \n" +
+        "                 </object> \n" +
+        "                </void> \n" +
+        "                <void method=\"add\"> \n" +
+        "                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "                  <void property=\"children\"> \n" +
+        "                   <object class=\"java.util.ArrayList\"> \n" +
+        "                    <void method=\"add\"> \n" +
+        "                     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "                      <void property=\"column\"> \n" +
+        "                       <string>first_name</string> \n" +
+        "                      </void> \n" +
+        "                      <void property=\"tabAlias\"> \n" +
+        "                       <string>orc_people</string> \n" +
+        "                      </void> \n" +
+        "                      <void property=\"typeInfo\"> \n" +
+        "                       <object id=\"PrimitiveTypeInfo2\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
+        "                        <void property=\"typeName\"> \n" +
+        "                         <string>string</string> \n" +
+        "                        </void> \n" +
+        "                       </object> \n" +
+        "                      </void> \n" +
+        "                     </object> \n" +
+        "                    </void> \n" +
+        "                    <void method=\"add\"> \n" +
+        "                     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "                      <void property=\"typeInfo\"> \n" +
+        "                       <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "                      </void> \n" +
+        "                      <void property=\"value\"> \n" +
+        "                       <string>alan</string> \n" +
+        "                      </void> \n" +
+        "                     </object> \n" +
+        "                    </void> \n" +
+        "                   </object> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"genericUDF\"> \n" +
+        "                   <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual\"/> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"typeInfo\"> \n" +
+        "                   <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "                  </void> \n" +
+        "                 </object> \n" +
+        "                </void> \n" +
+        "               </object> \n" +
+        "              </void> \n" +
+        "              <void property=\"genericUDF\"> \n" +
+        "               <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPAnd\"/> \n" +
+        "              </void> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "              <void property=\"children\"> \n" +
+        "               <object class=\"java.util.ArrayList\"> \n" +
+        "                <void method=\"add\"> \n" +
+        "                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "                  <void property=\"children\"> \n" +
+        "                   <object class=\"java.util.ArrayList\"> \n" +
+        "                    <void method=\"add\"> \n" +
+        "                     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "                      <void property=\"typeInfo\"> \n" +
+        "                       <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "                      </void> \n" +
+        "                      <void property=\"value\"> \n" +
+        "                       <string>xxxxx</string> \n" +
+        "                      </void> \n" +
+        "                     </object> \n" +
+        "                    </void> \n" +
+        "                    <void method=\"add\"> \n" +
+        "                     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "                      <void property=\"typeInfo\"> \n" +
+        "                       <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "                      </void> \n" +
+        "                      <void property=\"value\"> \n" +
+        "                       <int>3</int> \n" +
+        "                      </void> \n" +
+        "                     </object> \n" +
+        "                    </void> \n" +
+        "                   </object> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"genericUDF\"> \n" +
+        "                   <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge\"> \n" +
+        "                    <void property=\"udfClassName\"> \n" +
+        "                     <string>org.apache.hadoop.hive.ql.udf.UDFSubstr</string> \n" +
+        "                    </void> \n" +
+        "                    <void property=\"udfName\"> \n" +
+        "                     <string>substr</string> \n" +
+        "                    </void> \n" +
+        "                   </object> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"typeInfo\"> \n" +
+        "                   <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "                  </void> \n" +
+        "                 </object> \n" +
+        "                </void> \n" +
+        "                <void method=\"add\"> \n" +
+        "                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "                  <void property=\"column\"> \n" +
+        "                   <string>first_name</string> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"tabAlias\"> \n" +
+        "                   <string>orc_people</string> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"typeInfo\"> \n" +
+        "                   <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "                  </void> \n" +
+        "                 </object> \n" +
+        "                </void> \n" +
+        "               </object> \n" +
+        "              </void> \n" +
+        "              <void property=\"genericUDF\"> \n" +
+        "               <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual\"/> \n" +
+        "              </void> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "           </object> \n" +
+        "          </void> \n" +
+        "          <void property=\"genericUDF\"> \n" +
+        "           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPAnd\"/> \n" +
+        "          </void> \n" +
+        "          <void property=\"typeInfo\"> \n" +
+        "           <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "          </void> \n" +
+        "         </object> \n" +
+        "        </void> \n" +
+        "        <void method=\"add\"> \n" +
+        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "          <void property=\"children\"> \n" +
+        "           <object class=\"java.util.ArrayList\"> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "              </void> \n" +
+        "              <void property=\"value\"> \n" +
+        "               <string>smith</string> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "              <void property=\"column\"> \n" +
+        "               <string>last_name</string> \n" +
+        "              </void> \n" +
+        "              <void property=\"tabAlias\"> \n" +
+        "               <string>orc_people</string> \n" +
+        "              </void> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "           </object> \n" +
+        "          </void> \n" +
+        "          <void property=\"genericUDF\"> \n" +
+        "           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual\"/> \n" +
+        "          </void> \n" +
+        "          <void property=\"typeInfo\"> \n" +
+        "           <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "          </void> \n" +
+        "         </object> \n" +
+        "        </void> \n" +
+        "       </object> \n" +
+        "      </void> \n" +
+        "      <void property=\"genericUDF\"> \n" +
+        "       <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPAnd\"/> \n" +
+        "      </void> \n" +
+        "      <void property=\"typeInfo\"> \n" +
+        "       <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "      </void> \n" +
+        "     </object> \n" +
+        "    </void> \n" +
+        "    <void method=\"add\"> \n" +
+        "     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "      <void property=\"children\"> \n" +
+        "       <object class=\"java.util.ArrayList\"> \n" +
+        "        <void method=\"add\"> \n" +
+        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "          <void property=\"children\"> \n" +
+        "           <object class=\"java.util.ArrayList\"> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "              <void property=\"column\"> \n" +
+        "               <string>first_name</string> \n" +
+        "              </void> \n" +
+        "              <void property=\"tabAlias\"> \n" +
+        "               <string>orc_people</string> \n" +
+        "              </void> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "              </void> \n" +
+        "              <void property=\"value\"> \n" +
+        "               <int>3</int> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "           </object> \n" +
+        "          </void> \n" +
+        "          <void property=\"genericUDF\"> \n" +
+        "           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge\"> \n" +
+        "            <void property=\"udfClassName\"> \n" +
+        "             <string>org.apache.hadoop.hive.ql.udf.UDFSubstr</string> \n" +
+        "            </void> \n" +
+        "            <void property=\"udfName\"> \n" +
+        "             <string>substr</string> \n" +
+        "            </void> \n" +
+        "           </object> \n" +
+        "          </void> \n" +
+        "          <void property=\"typeInfo\"> \n" +
+        "           <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "          </void> \n" +
+        "         </object> \n" +
+        "        </void> \n" +
+        "        <void method=\"add\"> \n" +
+        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "          <void property=\"typeInfo\"> \n" +
+        "           <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "          </void> \n" +
+        "          <void property=\"value\"> \n" +
+        "           <string>yyy</string> \n" +
+        "          </void> \n" +
+        "         </object> \n" +
+        "        </void> \n" +
+        "       </object> \n" +
+        "      </void> \n" +
+        "      <void property=\"genericUDF\"> \n" +
+        "       <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual\"/> \n" +
+        "      </void> \n" +
+        "      <void property=\"typeInfo\"> \n" +
+        "       <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "      </void> \n" +
+        "     </object> \n" +
+        "    </void> \n" +
+        "   </object> \n" +
+        "  </void> \n" +
+        "  <void property=\"genericUDF\"> \n" +
+        "   <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPAnd\"/> \n" +
+        "  </void> \n" +
+        "  <void property=\"typeInfo\"> \n" +
+        "   <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "  </void> \n" +
+        " </object> \n" +
+        "</java> \n";
+
+    SearchArgumentImpl sarg =
+        (SearchArgumentImpl) ConvertAstToSearchArg.create(getFuncDesc(exprStr));
+    List<PredicateLeaf> leaves = sarg.getLeaves();
+    assertEquals(3, leaves.size());
+
+    String[] conditions = new String[]{
+      "lt(id, 45)",                         /* id between 23 and 45 */
+      "not(lteq(id, 23))",                   /* id between 23 and 45 */
+      "eq(first_name, Binary{\"alan\"})",   /* first_name = 'alan'  */
+      "eq(last_name, Binary{\"smith\"})"    /* 'smith' = last_name  */
+    };
+
+    FilterPredicate p = ParquetRecordReaderWrapper.toFilterPredicate(sarg);
+    String expected = String.format("and(and(and(%1$s, %2$s), %3$s), %4$s)", conditions);
+    assertEquals(expected, p.toString());
+
+    PredicateLeaf leaf = leaves.get(0);
+    assertEquals(PredicateLeaf.Type.INTEGER, leaf.getType());
+    assertEquals(PredicateLeaf.Operator.BETWEEN, leaf.getOperator());
+    assertEquals("id", leaf.getColumnName());
+    assertEquals(null, leaf.getLiteral());
+    assertEquals(23, leaf.getLiteralList().get(0));
+    assertEquals(45, leaf.getLiteralList().get(1));
+
+    leaf = leaves.get(1);
+    assertEquals(PredicateLeaf.Type.STRING, leaf.getType());
+    assertEquals(PredicateLeaf.Operator.EQUALS, leaf.getOperator());
+    assertEquals("first_name", leaf.getColumnName());
+    assertEquals("alan", leaf.getLiteral());
+
+    leaf = leaves.get(2);
+    assertEquals(PredicateLeaf.Type.STRING, leaf.getType());
+    assertEquals(PredicateLeaf.Operator.EQUALS, leaf.getOperator());
+    assertEquals("last_name", leaf.getColumnName());
+    assertEquals("smith", leaf.getLiteral());
+
+    assertEquals("(and leaf-0 leaf-1 leaf-2)",
+        sarg.getExpression().toString());
+    assertNoSharedNodes(sarg.getExpression(),
+        Sets.<ExpressionTree>newIdentityHashSet());
+  }
+
+  @Test
+  public void testExpression4() throws Exception {
+    /* id <> 12 and
+       first_name in ('john', 'sue') and
+       id in (34,50) */
+    String exprStr = "<?xml version=\"1.0\" encoding=\"UTF-8\"?> \n" +
+        "<java version=\"1.6.0_31\" class=\"java.beans.XMLDecoder\"> \n" +
+        " <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "  <void property=\"children\"> \n" +
+        "   <object class=\"java.util.ArrayList\"> \n" +
+        "    <void method=\"add\"> \n" +
+        "     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "      <void property=\"children\"> \n" +
+        "       <object class=\"java.util.ArrayList\"> \n" +
+        "        <void method=\"add\"> \n" +
+        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "          <void property=\"children\"> \n" +
+        "           <object class=\"java.util.ArrayList\"> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "              <void property=\"column\"> \n" +
+        "               <string>id</string> \n" +
+        "              </void> \n" +
+        "              <void property=\"tabAlias\"> \n" +
+        "               <string>orc_people</string> \n" +
+        "              </void> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object id=\"PrimitiveTypeInfo0\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
+        "                <void property=\"typeName\"> \n" +
+        "                 <string>int</string> \n" +
+        "                </void> \n" +
+        "               </object> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "              </void> \n" +
+        "              <void property=\"value\"> \n" +
+        "               <int>12</int> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "           </object> \n" +
+        "          </void> \n" +
+        "          <void property=\"genericUDF\"> \n" +
+        "           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNotEqual\"/> \n" +
+        "          </void> \n" +
+        "          <void property=\"typeInfo\"> \n" +
+        "           <object id=\"PrimitiveTypeInfo1\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
+        "            <void property=\"typeName\"> \n" +
+        "             <string>boolean</string> \n" +
+        "            </void> \n" +
+        "           </object> \n" +
+        "          </void> \n" +
+        "         </object> \n" +
+        "        </void> \n" +
+        "        <void method=\"add\"> \n" +
+        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "          <void property=\"children\"> \n" +
+        "           <object class=\"java.util.ArrayList\"> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "              <void property=\"column\"> \n" +
+        "               <string>first_name</string> \n" +
+        "              </void> \n" +
+        "              <void property=\"tabAlias\"> \n" +
+        "               <string>orc_people</string> \n" +
+        "              </void> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object id=\"PrimitiveTypeInfo2\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
+        "                <void property=\"typeName\"> \n" +
+        "                 <string>string</string> \n" +
+        "                </void> \n" +
+        "               </object> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "              </void> \n" +
+        "              <void property=\"value\"> \n" +
+        "               <string>john</string> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo2\"/> \n" +
+        "              </void> \n" +
+        "              <void property=\"value\"> \n" +
+        "               <string>sue</string> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "           </object> \n" +
+        "          </void> \n" +
+        "          <void property=\"genericUDF\"> \n" +
+        "           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFIn\"/> \n" +
+        "          </void> \n" +
+        "          <void property=\"typeInfo\"> \n" +
+        "           <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "          </void> \n" +
+        "         </object> \n" +
+        "        </void> \n" +
+        "       </object> \n" +
+        "      </void> \n" +
+        "      <void property=\"genericUDF\"> \n" +
+        "       <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPAnd\"/> \n" +
+        "      </void> \n" +
+        "      <void property=\"typeInfo\"> \n" +
+        "       <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "      </void> \n" +
+        "     </object> \n" +
+        "    </void> \n" +
+        "    <void method=\"add\"> \n" +
+        "     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "      <void property=\"children\"> \n" +
+        "       <object class=\"java.util.ArrayList\"> \n" +
+        "        <void method=\"add\"> \n" +
+        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "          <void property=\"column\"> \n" +
+        "           <string>id</string> \n" +
+        "          </void> \n" +
+        "          <void property=\"tabAlias\"> \n" +
+        "           <string>orc_people</string> \n" +
+        "          </void> \n" +
+        "          <void property=\"typeInfo\"> \n" +
+        "           <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "          </void> \n" +
+        "         </object> \n" +
+        "        </void> \n" +
+        "        <void method=\"add\"> \n" +
+        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "          <void property=\"typeInfo\"> \n" +
+        "           <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "          </void> \n" +
+        "          <void property=\"value\"> \n" +
+        "           <int>34</int> \n" +
+        "          </void> \n" +
+        "         </object> \n" +
+        "        </void> \n" +
+        "        <void method=\"add\"> \n" +
+        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "          <void property=\"typeInfo\"> \n" +
+        "           <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "          </void> \n" +
+        "          <void property=\"value\"> \n" +
+        "           <int>50</int> \n" +
+        "          </void> \n" +
+        "         </object> \n" +
+        "        </void> \n" +
+        "       </object> \n" +
+        "      </void> \n" +
+        "      <void property=\"genericUDF\"> \n" +
+        "       <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFIn\"/> \n" +
+        "      </void> \n" +
+        "      <void property=\"typeInfo\"> \n" +
+        "       <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "      </void> \n" +
+        "     </object> \n" +
+        "    </void> \n" +
+        "   </object> \n" +
+        "  </void> \n" +
+        "  <void property=\"genericUDF\"> \n" +
+        "   <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPAnd\"/> \n" +
+        "  </void> \n" +
+        "  <void property=\"typeInfo\"> \n" +
+        "   <object idref=\"PrimitiveTypeInfo1\"/> \n" +
+        "  </void> \n" +
+        " </object> \n" +
+        "</java> \n" +
+        "\n";
+
+    SearchArgumentImpl sarg =
+        (SearchArgumentImpl) ConvertAstToSearchArg.create(getFuncDesc(exprStr));
+    List<PredicateLeaf> leaves = sarg.getLeaves();
+    assertEquals(3, leaves.size());
+
+    String[] conditions = new String[]{
+      "not(eq(id, 12))", /* id <> 12 */
+      "or(eq(first_name, Binary{\"john\"}), eq(first_name, Binary{\"sue\"}))", /* first_name in
+      ('john', 'sue') */
+      "or(eq(id, 34), eq(id, 50))" /* id in (34,50) */
+    };
+
+    FilterPredicate p = ParquetRecordReaderWrapper.toFilterPredicate(sarg);
+    String expected = String.format("and(and(%1$s, %2$s), %3$s)", conditions);
+    assertEquals(expected, p.toString());
+
+    PredicateLeaf leaf = leaves.get(0);
+    assertEquals(PredicateLeaf.Type.INTEGER, leaf.getType());
+    assertEquals(PredicateLeaf.Operator.EQUALS, leaf.getOperator());
+    assertEquals("id", leaf.getColumnName());
+    assertEquals(12, leaf.getLiteral());
+
+    leaf = leaves.get(1);
+    assertEquals(PredicateLeaf.Type.STRING, leaf.getType());
+    assertEquals(PredicateLeaf.Operator.IN, leaf.getOperator());
+    assertEquals("first_name", leaf.getColumnName());
+    assertEquals("john", leaf.getLiteralList().get(0));
+    assertEquals("sue", leaf.getLiteralList().get(1));
+
+    leaf = leaves.get(2);
+    assertEquals(PredicateLeaf.Type.INTEGER, leaf.getType());
+    assertEquals(PredicateLeaf.Operator.IN, leaf.getOperator());
+    assertEquals("id", leaf.getColumnName());
+    assertEquals(34, leaf.getLiteralList().get(0));
+    assertEquals(50, leaf.getLiteralList().get(1));
+
+    assertEquals("(and (not leaf-0) leaf-1 leaf-2)",
+        sarg.getExpression().toString());
+    assertNoSharedNodes(sarg.getExpression(),
+        Sets.<ExpressionTree>newIdentityHashSet());
+    assertEquals(TruthValue.YES,
+        sarg.evaluate(values(TruthValue.NO, TruthValue.YES, TruthValue.YES)));
+    assertEquals(TruthValue.NULL,
+        sarg.evaluate(values(TruthValue.NULL, TruthValue.YES, TruthValue.YES)));
+    assertEquals(TruthValue.NULL,
+        sarg.evaluate(values(TruthValue.NO, TruthValue.NULL, TruthValue.YES)));
+    assertEquals(TruthValue.NO,
+        sarg.evaluate(values(TruthValue.YES, TruthValue.YES, TruthValue.YES)));
+    assertEquals(TruthValue.NO,
+        sarg.evaluate(values(TruthValue.NO, TruthValue.YES, TruthValue.NO)));
+    assertEquals(TruthValue.NO,
+        sarg.evaluate(values(TruthValue.NO, TruthValue.YES_NULL, TruthValue.NO)));
+    assertEquals(TruthValue.NO_NULL,
+        sarg.evaluate(values(TruthValue.NO, TruthValue.NULL, TruthValue.YES_NO_NULL)));
+    assertEquals(TruthValue.NO_NULL,
+        sarg.evaluate(values(TruthValue.NO, TruthValue.YES, TruthValue.NO_NULL)));
+  }
+
+  @Test
+  public void testExpression5() throws Exception {
+    /* (first_name < 'owen' or 'foobar' = substr(last_name, 4)) and
+    first_name between 'david' and 'greg' */
+    String exprStr = "<?xml version=\"1.0\" encoding=\"UTF-8\"?> \n" +
+        "<java version=\"1.6.0_31\" class=\"java.beans.XMLDecoder\"> \n" +
+        " <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "  <void property=\"children\"> \n" +
+        "   <object class=\"java.util.ArrayList\"> \n" +
+        "    <void method=\"add\"> \n" +
+        "     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "      <void property=\"children\"> \n" +
+        "       <object class=\"java.util.ArrayList\"> \n" +
+        "        <void method=\"add\"> \n" +
+        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "          <void property=\"children\"> \n" +
+        "           <object class=\"java.util.ArrayList\"> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "              <void property=\"column\"> \n" +
+        "               <string>first_name</string> \n" +
+        "              </void> \n" +
+        "              <void property=\"tabAlias\"> \n" +
+        "               <string>orc_people</string> \n" +
+        "              </void> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object id=\"PrimitiveTypeInfo0\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
+        "                <void property=\"typeName\"> \n" +
+        "                 <string>string</string> \n" +
+        "                </void> \n" +
+        "               </object> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "              </void> \n" +
+        "              <void property=\"value\"> \n" +
+        "               <string>owen</string> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "           </object> \n" +
+        "          </void> \n" +
+        "          <void property=\"genericUDF\"> \n" +
+        "           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPLessThan\"/> \n" +
+        "          </void> \n" +
+        "          <void property=\"typeInfo\"> \n" +
+        "           <object id=\"PrimitiveTypeInfo1\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
+        "            <void property=\"typeName\"> \n" +
+        "             <string>boolean</string> \n" +
+        "            </void> \n" +
+        "           </object> \n" +
+        "          </void> \n" +
+        "         </object> \n" +
+        "        </void> \n" +
+        "        <void method=\"add\"> \n" +
+        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "          <void property=\"children\"> \n" +
+        "           <object class=\"java.util.ArrayList\"> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "              <void property=\"typeInfo\"> \n" +
+        "               <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "              </void> \n" +
+        "              <void property=\"value\"> \n" +
+        "               <string>foobar</string> \n" +
+        "              </void> \n" +
+        "             </object> \n" +
+        "            </void> \n" +
+        "            <void method=\"add\"> \n" +
+        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
+        "              <void property=\"children\"> \n" +
+        "               <object class=\"java.util.ArrayList\"> \n" +
+        "                <void method=\"add\"> \n" +
+        "                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
+        "                  <void property=\"column\"> \n" +
+        "                   <string>last_name</string> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"tabAlias\"> \n" +
+        "                   <string>orc_people</string> \n" +
+        "                  </void> \n" +
+        "                  <void property=\"typeInfo\"> \n" +
+        "                   <object idref=\"PrimitiveTypeInfo0\"/> \n" +
+        "                  </void> \n" +
+        "                 </object> \n" +
+        "                </void> \n" +
+        "                <void method=\"add\"> \n" +
+        "                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
+        "                  <void property=\"typeInfo\"> \n" +
+        "          

<TRUNCATED>

[18/43] hive git commit: HIVE-11333: ColumnPruner prunes columns of UnionOperator that should be kept (Pengcheng Xiong, via Jesus Camacho Rodriguez)

Posted by xu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/2773b9d9/ql/src/test/results/clientpositive/union16.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union16.q.out b/ql/src/test/results/clientpositive/union16.q.out
index f503440..9f7dd68 100644
--- a/ql/src/test/results/clientpositive/union16.q.out
+++ b/ql/src/test/results/clientpositive/union16.q.out
@@ -74,463 +74,513 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: src
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+              expressions: value (type: string)
+              outputColumnNames: _col0
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Union
-                Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                  Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     mode: hash
                     outputColumnNames: _col0
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    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: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col0 (type: bigint)
           TableScan
             alias: src
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+              expressions: value (type: string)
+              outputColumnNames: _col0
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Union
-                Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                  Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     mode: hash
                     outputColumnNames: _col0
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    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: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col0 (type: bigint)
           TableScan
             alias: src
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+              expressions: value (type: string)
+              outputColumnNames: _col0
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Union
-                Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                  Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     mode: hash
                     outputColumnNames: _col0
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    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: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col0 (type: bigint)
           TableScan
             alias: src
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+              expressions: value (type: string)
+              outputColumnNames: _col0
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Union
-                Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                  Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     mode: hash
                     outputColumnNames: _col0
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    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: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col0 (type: bigint)
           TableScan
             alias: src
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+              expressions: value (type: string)
+              outputColumnNames: _col0
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Union
-                Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                  Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     mode: hash
                     outputColumnNames: _col0
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    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: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col0 (type: bigint)
           TableScan
             alias: src
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+              expressions: value (type: string)
+              outputColumnNames: _col0
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Union
-                Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                  Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     mode: hash
                     outputColumnNames: _col0
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    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: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col0 (type: bigint)
           TableScan
             alias: src
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+              expressions: value (type: string)
+              outputColumnNames: _col0
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Union
-                Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                  Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     mode: hash
                     outputColumnNames: _col0
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    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: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col0 (type: bigint)
           TableScan
             alias: src
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+              expressions: value (type: string)
+              outputColumnNames: _col0
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Union
-                Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                  Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     mode: hash
                     outputColumnNames: _col0
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    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: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col0 (type: bigint)
           TableScan
             alias: src
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+              expressions: value (type: string)
+              outputColumnNames: _col0
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Union
-                Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                  Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     mode: hash
                     outputColumnNames: _col0
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    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: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col0 (type: bigint)
           TableScan
             alias: src
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+              expressions: value (type: string)
+              outputColumnNames: _col0
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Union
-                Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                  Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     mode: hash
                     outputColumnNames: _col0
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    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: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col0 (type: bigint)
           TableScan
             alias: src
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+              expressions: value (type: string)
+              outputColumnNames: _col0
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Union
-                Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                  Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     mode: hash
                     outputColumnNames: _col0
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    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: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col0 (type: bigint)
           TableScan
             alias: src
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+              expressions: value (type: string)
+              outputColumnNames: _col0
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Union
-                Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                  Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     mode: hash
                     outputColumnNames: _col0
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    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: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col0 (type: bigint)
           TableScan
             alias: src
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+              expressions: value (type: string)
+              outputColumnNames: _col0
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Union
-                Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                  Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     mode: hash
                     outputColumnNames: _col0
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    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: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col0 (type: bigint)
           TableScan
             alias: src
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+              expressions: value (type: string)
+              outputColumnNames: _col0
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Union
-                Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                  Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     mode: hash
                     outputColumnNames: _col0
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    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: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col0 (type: bigint)
           TableScan
             alias: src
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+              expressions: value (type: string)
+              outputColumnNames: _col0
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Union
-                Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                  Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     mode: hash
                     outputColumnNames: _col0
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    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: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col0 (type: bigint)
           TableScan
             alias: src
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+              expressions: value (type: string)
+              outputColumnNames: _col0
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Union
-                Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                  Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     mode: hash
                     outputColumnNames: _col0
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    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: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col0 (type: bigint)
           TableScan
             alias: src
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+              expressions: value (type: string)
+              outputColumnNames: _col0
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Union
-                Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                  Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     mode: hash
                     outputColumnNames: _col0
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    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: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col0 (type: bigint)
           TableScan
             alias: src
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+              expressions: value (type: string)
+              outputColumnNames: _col0
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Union
-                Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                  Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     mode: hash
                     outputColumnNames: _col0
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    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: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col0 (type: bigint)
           TableScan
             alias: src
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+              expressions: value (type: string)
+              outputColumnNames: _col0
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Union
-                Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                  Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     mode: hash
                     outputColumnNames: _col0
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    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: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col0 (type: bigint)
           TableScan
             alias: src
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+              expressions: value (type: string)
+              outputColumnNames: _col0
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Union
-                Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                  Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     mode: hash
                     outputColumnNames: _col0
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    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: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col0 (type: bigint)
           TableScan
             alias: src
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+              expressions: value (type: string)
+              outputColumnNames: _col0
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Union
-                Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                  Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     mode: hash
                     outputColumnNames: _col0
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    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: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col0 (type: bigint)
           TableScan
             alias: src
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+              expressions: value (type: string)
+              outputColumnNames: _col0
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Union
-                Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                  Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     mode: hash
                     outputColumnNames: _col0
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    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: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col0 (type: bigint)
           TableScan
             alias: src
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+              expressions: value (type: string)
+              outputColumnNames: _col0
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Union
-                Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                  Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     mode: hash
                     outputColumnNames: _col0
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    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: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col0 (type: bigint)
           TableScan
             alias: src
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+              expressions: value (type: string)
+              outputColumnNames: _col0
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Union
-                Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                  Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     mode: hash
                     outputColumnNames: _col0
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    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: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col0 (type: bigint)
           TableScan
             alias: src
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+              expressions: value (type: string)
+              outputColumnNames: _col0
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Union
-                Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                  Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     mode: hash
                     outputColumnNames: _col0
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    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: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col0 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
           mode: mergepartial
           outputColumnNames: _col0
-          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
             table:
                 input format: org.apache.hadoop.mapred.TextInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/2773b9d9/ql/src/test/results/clientpositive/union2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union2.q.out b/ql/src/test/results/clientpositive/union2.q.out
index 5007d01..69559a4 100644
--- a/ql/src/test/results/clientpositive/union2.q.out
+++ b/ql/src/test/results/clientpositive/union2.q.out
@@ -22,49 +22,53 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: s1
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+              expressions: value (type: string)
+              outputColumnNames: _col0
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Union
-                Statistics: Num rows: 1000 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  Statistics: Num rows: 1000 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                  Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     mode: hash
                     outputColumnNames: _col0
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    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: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col0 (type: bigint)
           TableScan
             alias: s1
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+              expressions: value (type: string)
+              outputColumnNames: _col0
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Union
-                Statistics: Num rows: 1000 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  Statistics: Num rows: 1000 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                  Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     mode: hash
                     outputColumnNames: _col0
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    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: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col0 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
           mode: mergepartial
           outputColumnNames: _col0
-          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
             table:
                 input format: org.apache.hadoop.mapred.TextInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/2773b9d9/ql/src/test/results/clientpositive/union9.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union9.q.out b/ql/src/test/results/clientpositive/union9.q.out
index 5f54210..ec7ab35 100644
--- a/ql/src/test/results/clientpositive/union9.q.out
+++ b/ql/src/test/results/clientpositive/union9.q.out
@@ -24,67 +24,73 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: s1
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+              expressions: value (type: string)
+              outputColumnNames: _col0
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Union
-                Statistics: Num rows: 1500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                Statistics: Num rows: 1500 Data size: 15936 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  Statistics: Num rows: 1500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                  Statistics: Num rows: 1500 Data size: 15936 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     mode: hash
                     outputColumnNames: _col0
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    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: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col0 (type: bigint)
           TableScan
             alias: s1
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+              expressions: value (type: string)
+              outputColumnNames: _col0
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Union
-                Statistics: Num rows: 1500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                Statistics: Num rows: 1500 Data size: 15936 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  Statistics: Num rows: 1500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                  Statistics: Num rows: 1500 Data size: 15936 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     mode: hash
                     outputColumnNames: _col0
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    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: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col0 (type: bigint)
           TableScan
             alias: s1
-            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+              expressions: value (type: string)
+              outputColumnNames: _col0
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Union
-                Statistics: Num rows: 1500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                Statistics: Num rows: 1500 Data size: 15936 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
-                  Statistics: Num rows: 1500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                  Statistics: Num rows: 1500 Data size: 15936 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count(1)
                     mode: hash
                     outputColumnNames: _col0
-                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    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: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col0 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
           mode: mergepartial
           outputColumnNames: _col0
-          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
-            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
             table:
                 input format: org.apache.hadoop.mapred.TextInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/2773b9d9/ql/src/test/results/clientpositive/union_view.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union_view.q.out b/ql/src/test/results/clientpositive/union_view.q.out
index 66ca51b..c55dee6 100644
--- a/ql/src/test/results/clientpositive/union_view.q.out
+++ b/ql/src/test/results/clientpositive/union_view.q.out
@@ -872,6 +872,8 @@ STAGE PLANS:
             filterExpr: (ds = '1') (type: boolean)
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
+              expressions: '1' (type: string)
+              outputColumnNames: _col0
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Union
                 Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -894,6 +896,8 @@ STAGE PLANS:
               predicate: (ds = '1') (type: boolean)
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Select Operator
+                expressions: '1' (type: string)
+                outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                 Union
                   Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -916,6 +920,8 @@ STAGE PLANS:
               predicate: (ds = '1') (type: boolean)
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Select Operator
+                expressions: '1' (type: string)
+                outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                 Union
                   Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -990,6 +996,8 @@ STAGE PLANS:
               predicate: (ds = '2') (type: boolean)
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Select Operator
+                expressions: '2' (type: string)
+                outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                 Union
                   Statistics: Num rows: 1002 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
@@ -1009,6 +1017,8 @@ STAGE PLANS:
             filterExpr: (ds = '2') (type: boolean)
             Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
             Select Operator
+              expressions: '2' (type: string)
+              outputColumnNames: _col0
               Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
               Union
                 Statistics: Num rows: 1002 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
@@ -1031,6 +1041,8 @@ STAGE PLANS:
               predicate: (ds = '2') (type: boolean)
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Select Operator
+                expressions: '2' (type: string)
+                outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                 Union
                   Statistics: Num rows: 1002 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
@@ -1105,6 +1117,8 @@ STAGE PLANS:
               predicate: (ds = '3') (type: boolean)
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Select Operator
+                expressions: '3' (type: string)
+                outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                 Union
                   Statistics: Num rows: 1002 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
@@ -1127,6 +1141,8 @@ STAGE PLANS:
               predicate: (ds = '3') (type: boolean)
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Select Operator
+                expressions: '3' (type: string)
+                outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                 Union
                   Statistics: Num rows: 1002 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
@@ -1146,6 +1162,8 @@ STAGE PLANS:
             filterExpr: (ds = '3') (type: boolean)
             Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
             Select Operator
+              expressions: '3' (type: string)
+              outputColumnNames: _col0
               Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
               Union
                 Statistics: Num rows: 1002 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
@@ -1337,6 +1355,8 @@ STAGE PLANS:
               predicate: (ds = '4') (type: boolean)
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Select Operator
+                expressions: '4' (type: string)
+                outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                 Union
                   Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -1359,6 +1379,8 @@ STAGE PLANS:
               predicate: (ds = '4') (type: boolean)
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Select Operator
+                expressions: '4' (type: string)
+                outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                 Union
                   Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -1378,6 +1400,8 @@ STAGE PLANS:
             filterExpr: (ds = '4') (type: boolean)
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Select Operator
+              expressions: '4' (type: string)
+              outputColumnNames: _col0
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Union
                 Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/2773b9d9/ql/src/test/results/clientpositive/vector_null_projection.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_null_projection.q.out b/ql/src/test/results/clientpositive/vector_null_projection.q.out
index c787fd4..b3b44ad 100644
--- a/ql/src/test/results/clientpositive/vector_null_projection.q.out
+++ b/ql/src/test/results/clientpositive/vector_null_projection.q.out
@@ -100,6 +100,8 @@ STAGE PLANS:
             alias: a
             Statistics: Num rows: 1 Data size: 87 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
+              expressions: null (type: void)
+              outputColumnNames: _col0
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
               Union
                 Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
@@ -119,6 +121,8 @@ STAGE PLANS:
             alias: b
             Statistics: Num rows: 1 Data size: 87 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
+              expressions: null (type: void)
+              outputColumnNames: _col0
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
               Union
                 Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE


[05/43] hive git commit: HIVE-11229 Mutation API: Coordinator communication with meta store should be optional (Elliot West via gates)

Posted by xu...@apache.org.
HIVE-11229 Mutation API: Coordinator communication with meta store should be optional (Elliot West via gates)


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

Branch: refs/heads/spark
Commit: 6ec72de79ebb898f699402e8a2d7681c4e39ecd2
Parents: dfdc670
Author: Alan Gates <ga...@hortonworks.com>
Authored: Tue Jul 21 11:08:59 2015 -0700
Committer: Alan Gates <ga...@hortonworks.com>
Committed: Tue Jul 21 11:08:59 2015 -0700

----------------------------------------------------------------------
 .../hive/hcatalog/streaming/mutate/package.html |  31 ++++-
 .../mutate/worker/CreatePartitionHelper.java    |  83 --------------
 .../mutate/worker/MetaStorePartitionHelper.java | 102 +++++++++++++++++
 .../mutate/worker/MutatorCoordinator.java       |  21 ++--
 .../worker/MutatorCoordinatorBuilder.java       |  41 +++++--
 .../mutate/worker/PartitionHelper.java          |  17 +++
 .../mutate/worker/WarehousePartitionHelper.java |  69 ++++++++++++
 .../worker/TestMetaStorePartitionHelper.java    | 112 +++++++++++++++++++
 .../mutate/worker/TestMutatorCoordinator.java   |  40 ++++---
 .../worker/TestWarehousePartitionHelper.java    |  57 ++++++++++
 10 files changed, 452 insertions(+), 121 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/6ec72de7/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/package.html
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/package.html b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/package.html
index 09a55b6..72ce6b1 100644
--- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/package.html
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/package.html
@@ -416,17 +416,39 @@ data, is the responsibility of the client using the API.
 </p>
 
 <h3>Dynamic Partition Creation:</h3>
+<p>
 It is very likely to be desirable to have new partitions created
 automatically (say on a hourly basis). In such cases requiring the Hive
-admin to pre-create the necessary partitions may not be reasonable.
-Consequently the API allows coordinators to create partitions as needed
-(see:
+admin to pre-create the necessary partitions may not be reasonable. The
+API allows coordinators to create partitions as needed (see:
 <code>MutatorClientBuilder.addSinkTable(String, String, boolean)</code>
 ). Partition creation being an atomic action, multiple coordinators can
 race to create the partition, but only one would succeed, so
 coordinators clients need not synchronize when creating a partition. The
 user of the coordinator process needs to be given write permissions on
 the Hive table in order to create partitions.
+</p>
+
+<p>Care must be taken when using this option as it requires that the
+coordinators maintain a connection with the meta store database. When
+coordinator are running in a distributed environment (as is likely the
+case) it possible for them to overwhelm the meta store. In such cases it
+may be better to disable partition creation and collect a set of
+affected partitions as part of your ETL merge process. These can then be
+created with a single meta store connection in your client code, once
+the cluster side merge process is complete.</p>
+<p>
+Finally, note that when partition creation is disabled the coordinators
+must synthesize the partition URI as they cannot retrieve it from the
+meta store. This may cause problems if the layout of your partitions in
+HDFS does not follow the Hive standard (as implemented in
+<code>
+org.apache.hadoop.hive.metastore.Warehouse.getPartitionPath(Path,
+LinkedHashMap
+&lt;String , String&gt;).
+</code>
+)
+</p>
 
 <h2>Reading data</h2>
 
@@ -473,6 +495,7 @@ table. The <code>AcidTableSerializer</code> can help you transport the <code>Aci
 when your workers are in a distributed environment.
 </li>
 <li>Compute your mutation set (this is your ETL merge process).</li>
+<li>Optionally: collect the set of affected partitions.</li>
 <li>Append bucket ids to insertion records. A <code>BucketIdResolver</code>
 can help here.
 </li>
@@ -481,6 +504,8 @@ can help here.
 <li>Close your coordinators.</li>
 <li>Abort or commit the transaction.</li>
 <li>Close your mutation client.</li>
+<li>Optionally: create any affected partitions that do not exist in
+the meta store.</li>
 </ol>
 <p>
 See

http://git-wip-us.apache.org/repos/asf/hive/blob/6ec72de7/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/CreatePartitionHelper.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/CreatePartitionHelper.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/CreatePartitionHelper.java
deleted file mode 100644
index 9aab346..0000000
--- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/CreatePartitionHelper.java
+++ /dev/null
@@ -1,83 +0,0 @@
-package org.apache.hive.hcatalog.streaming.mutate.worker;
-
-import java.util.List;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.metastore.IMetaStoreClient;
-import org.apache.hadoop.hive.metastore.Warehouse;
-import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
-import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
-import org.apache.hadoop.hive.metastore.api.Partition;
-import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
-import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.thrift.TException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/** Utility class that can create new table partitions within the {@link IMetaStoreClient meta store}. */
-class CreatePartitionHelper {
-
-  private static final Logger LOG = LoggerFactory.getLogger(CreatePartitionHelper.class);
-
-  private final IMetaStoreClient metaStoreClient;
-  private final String databaseName;
-  private final String tableName;
-
-  CreatePartitionHelper(IMetaStoreClient metaStoreClient, String databaseName, String tableName) {
-    this.metaStoreClient = metaStoreClient;
-    this.databaseName = databaseName;
-    this.tableName = tableName;
-  }
-
-  /** Returns the expected {@link Path} for a given partition value. */
-  Path getPathForPartition(List<String> newPartitionValues) throws WorkerException {
-    try {
-      String location;
-      if (newPartitionValues.isEmpty()) {
-        location = metaStoreClient.getTable(databaseName, tableName).getSd().getLocation();
-      } else {
-        location = metaStoreClient.getPartition(databaseName, tableName, newPartitionValues).getSd().getLocation();
-      }
-      LOG.debug("Found path {} for partition {}", location, newPartitionValues);
-      return new Path(location);
-    } catch (NoSuchObjectException e) {
-      throw new WorkerException("Table not found '" + databaseName + "." + tableName + "'.", e);
-    } catch (TException e) {
-      throw new WorkerException("Failed to get path for partitions '" + newPartitionValues + "' on table '"
-          + databaseName + "." + tableName + "' with meta store: " + metaStoreClient, e);
-    }
-  }
-
-  /** Creates the specified partition if it does not already exist. Does nothing if the table is unpartitioned. */
-  void createPartitionIfNotExists(List<String> newPartitionValues) throws WorkerException {
-    if (newPartitionValues.isEmpty()) {
-      return;
-    }
-
-    try {
-      LOG.debug("Attempting to create partition (if not exists) {}.{}:{}", databaseName, tableName, newPartitionValues);
-      Table table = metaStoreClient.getTable(databaseName, tableName);
-
-      Partition partition = new Partition();
-      partition.setDbName(table.getDbName());
-      partition.setTableName(table.getTableName());
-      StorageDescriptor partitionSd = new StorageDescriptor(table.getSd());
-      partitionSd.setLocation(table.getSd().getLocation() + Path.SEPARATOR
-          + Warehouse.makePartName(table.getPartitionKeys(), newPartitionValues));
-      partition.setSd(partitionSd);
-      partition.setValues(newPartitionValues);
-
-      metaStoreClient.add_partition(partition);
-    } catch (AlreadyExistsException e) {
-      LOG.debug("Partition already exisits: {}.{}:{}", databaseName, tableName, newPartitionValues);
-    } catch (NoSuchObjectException e) {
-      LOG.error("Failed to create partition : " + newPartitionValues, e);
-      throw new PartitionCreationException("Table not found '" + databaseName + "." + tableName + "'.", e);
-    } catch (TException e) {
-      LOG.error("Failed to create partition : " + newPartitionValues, e);
-      throw new PartitionCreationException("Failed to create partition '" + newPartitionValues + "' on table '"
-          + databaseName + "." + tableName + "'", e);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/6ec72de7/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MetaStorePartitionHelper.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MetaStorePartitionHelper.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MetaStorePartitionHelper.java
new file mode 100644
index 0000000..7e2e006
--- /dev/null
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MetaStorePartitionHelper.java
@@ -0,0 +1,102 @@
+package org.apache.hive.hcatalog.streaming.mutate.worker;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.Warehouse;
+import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A {@link PartitionHelper} implementation that uses the {@link IMetaStoreClient meta store} to both create partitions
+ * and obtain information concerning partitions. Exercise care when using this from within workers that are running in a
+ * cluster as it may overwhelm the meta store database instance. As an alternative, consider using the
+ * {@link WarehousePartitionHelper}, collecting the affected partitions as an output of your merge job, and then
+ * retrospectively adding partitions in your client.
+ */
+class MetaStorePartitionHelper implements PartitionHelper {
+
+  private static final Logger LOG = LoggerFactory.getLogger(MetaStorePartitionHelper.class);
+
+  private final IMetaStoreClient metaStoreClient;
+  private final String databaseName;
+  private final String tableName;
+  private final Path tablePath;
+
+  MetaStorePartitionHelper(IMetaStoreClient metaStoreClient, String databaseName, String tableName, Path tablePath) {
+    this.metaStoreClient = metaStoreClient;
+    this.tablePath = tablePath;
+    this.databaseName = databaseName;
+    this.tableName = tableName;
+  }
+
+  /** Returns the expected {@link Path} for a given partition value. */
+  @Override
+  public Path getPathForPartition(List<String> newPartitionValues) throws WorkerException {
+    if (newPartitionValues.isEmpty()) {
+      LOG.debug("Using path {} for unpartitioned table {}.{}", tablePath, databaseName, tableName);
+      return tablePath;
+    } else {
+      try {
+        String location = metaStoreClient
+            .getPartition(databaseName, tableName, newPartitionValues)
+            .getSd()
+            .getLocation();
+        LOG.debug("Found path {} for partition {}", location, newPartitionValues);
+        return new Path(location);
+      } catch (NoSuchObjectException e) {
+        throw new WorkerException("Table not found '" + databaseName + "." + tableName + "'.", e);
+      } catch (TException e) {
+        throw new WorkerException("Failed to get path for partitions '" + newPartitionValues + "' on table '"
+            + databaseName + "." + tableName + "' with meta store: " + metaStoreClient, e);
+      }
+    }
+  }
+
+  /** Creates the specified partition if it does not already exist. Does nothing if the table is unpartitioned. */
+  @Override
+  public void createPartitionIfNotExists(List<String> newPartitionValues) throws WorkerException {
+    if (newPartitionValues.isEmpty()) {
+      return;
+    }
+
+    try {
+      LOG.debug("Attempting to create partition (if not exists) {}.{}:{}", databaseName, tableName, newPartitionValues);
+      Table table = metaStoreClient.getTable(databaseName, tableName);
+
+      Partition partition = new Partition();
+      partition.setDbName(table.getDbName());
+      partition.setTableName(table.getTableName());
+      StorageDescriptor partitionSd = new StorageDescriptor(table.getSd());
+      partitionSd.setLocation(table.getSd().getLocation() + Path.SEPARATOR
+          + Warehouse.makePartName(table.getPartitionKeys(), newPartitionValues));
+      partition.setSd(partitionSd);
+      partition.setValues(newPartitionValues);
+
+      metaStoreClient.add_partition(partition);
+    } catch (AlreadyExistsException e) {
+      LOG.debug("Partition already exisits: {}.{}:{}", databaseName, tableName, newPartitionValues);
+    } catch (NoSuchObjectException e) {
+      LOG.error("Failed to create partition : " + newPartitionValues, e);
+      throw new PartitionCreationException("Table not found '" + databaseName + "." + tableName + "'.", e);
+    } catch (TException e) {
+      LOG.error("Failed to create partition : " + newPartitionValues, e);
+      throw new PartitionCreationException("Failed to create partition '" + newPartitionValues + "' on table '"
+          + databaseName + "." + tableName + "'", e);
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    metaStoreClient.close();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/6ec72de7/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorCoordinator.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorCoordinator.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorCoordinator.java
index 96f05e5..eaed09e 100644
--- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorCoordinator.java
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorCoordinator.java
@@ -11,7 +11,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.JavaUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.ql.io.AcidOutputFormat;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.io.RecordIdentifier;
@@ -40,13 +39,12 @@ public class MutatorCoordinator implements Closeable, Flushable {
 
   private static final Logger LOG = LoggerFactory.getLogger(MutatorCoordinator.class);
 
-  private final IMetaStoreClient metaStoreClient;
   private final MutatorFactory mutatorFactory;
   private final GroupingValidator groupingValidator;
   private final SequenceValidator sequenceValidator;
   private final AcidTable table;
   private final RecordInspector recordInspector;
-  private final CreatePartitionHelper partitionHelper;
+  private final PartitionHelper partitionHelper;
   private final AcidOutputFormat<?, ?> outputFormat;
   private final BucketIdResolver bucketIdResolver;
   private final HiveConf configuration;
@@ -57,18 +55,16 @@ public class MutatorCoordinator implements Closeable, Flushable {
   private Path partitionPath;
   private Mutator mutator;
 
-  MutatorCoordinator(IMetaStoreClient metaStoreClient, HiveConf configuration, MutatorFactory mutatorFactory,
+  MutatorCoordinator(HiveConf configuration, MutatorFactory mutatorFactory, PartitionHelper partitionHelper,
       AcidTable table, boolean deleteDeltaIfExists) throws WorkerException {
-    this(metaStoreClient, configuration, mutatorFactory, new CreatePartitionHelper(metaStoreClient,
-        table.getDatabaseName(), table.getTableName()), new GroupingValidator(), new SequenceValidator(), table,
+    this(configuration, mutatorFactory, partitionHelper, new GroupingValidator(), new SequenceValidator(), table,
         deleteDeltaIfExists);
   }
 
   /** Visible for testing only. */
-  MutatorCoordinator(IMetaStoreClient metaStoreClient, HiveConf configuration, MutatorFactory mutatorFactory,
-      CreatePartitionHelper partitionHelper, GroupingValidator groupingValidator, SequenceValidator sequenceValidator,
-      AcidTable table, boolean deleteDeltaIfExists) throws WorkerException {
-    this.metaStoreClient = metaStoreClient;
+  MutatorCoordinator(HiveConf configuration, MutatorFactory mutatorFactory, PartitionHelper partitionHelper,
+      GroupingValidator groupingValidator, SequenceValidator sequenceValidator, AcidTable table,
+      boolean deleteDeltaIfExists) throws WorkerException {
     this.configuration = configuration;
     this.mutatorFactory = mutatorFactory;
     this.partitionHelper = partitionHelper;
@@ -156,7 +152,7 @@ public class MutatorCoordinator implements Closeable, Flushable {
         mutator.close();
       }
     } finally {
-      metaStoreClient.close();
+      partitionHelper.close();
     }
   }
 
@@ -178,7 +174,7 @@ public class MutatorCoordinator implements Closeable, Flushable {
 
     try {
       if (partitionHasChanged(newPartitionValues)) {
-        if (table.createPartitions()) {
+        if (table.createPartitions() && operationType == OperationType.INSERT) {
           partitionHelper.createPartitionIfNotExists(newPartitionValues);
         }
         Path newPartitionPath = partitionHelper.getPathForPartition(newPartitionValues);
@@ -265,6 +261,7 @@ public class MutatorCoordinator implements Closeable, Flushable {
     }
   }
 
+  /* A delta may be present from a previous failed task attempt. */
   private void deleteDeltaIfExists(Path partitionPath, long transactionId, int bucketId) throws IOException {
     Path deltaPath = AcidUtils.createFilename(partitionPath,
         new AcidOutputFormat.Options(configuration)

http://git-wip-us.apache.org/repos/asf/hive/blob/6ec72de7/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorCoordinatorBuilder.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorCoordinatorBuilder.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorCoordinatorBuilder.java
index 8851ea6..cd28e02 100644
--- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorCoordinatorBuilder.java
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorCoordinatorBuilder.java
@@ -1,9 +1,13 @@
 package org.apache.hive.hcatalog.streaming.mutate.worker;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
 
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hive.hcatalog.common.HCatUtil;
@@ -57,20 +61,41 @@ public class MutatorCoordinatorBuilder {
   }
 
   public MutatorCoordinator build() throws WorkerException, MetaException {
-    String user = authenticatedUser == null ? System.getProperty("user.name") : authenticatedUser.getShortUserName();
-    boolean secureMode = authenticatedUser == null ? false : authenticatedUser.hasKerberosCredentials();
-
     configuration = HiveConfFactory.newInstance(configuration, this.getClass(), metaStoreUri);
 
-    IMetaStoreClient metaStoreClient;
+    PartitionHelper partitionHelper;
+    if (table.createPartitions()) {
+      partitionHelper = newMetaStorePartitionHelper();
+    } else {
+      partitionHelper = newWarehousePartitionHelper();
+    }
+
+    return new MutatorCoordinator(configuration, mutatorFactory, partitionHelper, table, deleteDeltaIfExists);
+  }
+
+  private PartitionHelper newWarehousePartitionHelper() throws MetaException, WorkerException {
+    String location = table.getTable().getSd().getLocation();
+    Path tablePath = new Path(location);
+    List<FieldSchema> partitionFields = table.getTable().getPartitionKeys();
+    List<String> partitionColumns = new ArrayList<>(partitionFields.size());
+    for (FieldSchema field : partitionFields) {
+      partitionColumns.add(field.getName());
+    }
+    return new WarehousePartitionHelper(configuration, tablePath, partitionColumns);
+  }
+
+  private PartitionHelper newMetaStorePartitionHelper() throws MetaException, WorkerException {
+    String user = authenticatedUser == null ? System.getProperty("user.name") : authenticatedUser.getShortUserName();
+    boolean secureMode = authenticatedUser == null ? false : authenticatedUser.hasKerberosCredentials();
     try {
-      metaStoreClient = new UgiMetaStoreClientFactory(metaStoreUri, configuration, authenticatedUser, user, secureMode)
-          .newInstance(HCatUtil.getHiveMetastoreClient(configuration));
+      IMetaStoreClient metaStoreClient = new UgiMetaStoreClientFactory(metaStoreUri, configuration, authenticatedUser,
+          user, secureMode).newInstance(HCatUtil.getHiveMetastoreClient(configuration));
+      String tableLocation = table.getTable().getSd().getLocation();
+      Path tablePath = new Path(tableLocation);
+      return new MetaStorePartitionHelper(metaStoreClient, table.getDatabaseName(), table.getTableName(), tablePath);
     } catch (IOException e) {
       throw new WorkerException("Could not create meta store client.", e);
     }
-
-    return new MutatorCoordinator(metaStoreClient, configuration, mutatorFactory, table, deleteDeltaIfExists);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/6ec72de7/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/PartitionHelper.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/PartitionHelper.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/PartitionHelper.java
new file mode 100644
index 0000000..d70207a
--- /dev/null
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/PartitionHelper.java
@@ -0,0 +1,17 @@
+package org.apache.hive.hcatalog.streaming.mutate.worker;
+
+import java.io.Closeable;
+import java.util.List;
+
+import org.apache.hadoop.fs.Path;
+
+/** Implementations are responsible for creating and obtaining path information about partitions. */
+interface PartitionHelper extends Closeable {
+
+  /** Return the location of the partition described by the provided values. */
+  Path getPathForPartition(List<String> newPartitionValues) throws WorkerException;
+
+  /** Create the partition described by the provided values if it does not exist already. */
+  void createPartitionIfNotExists(List<String> newPartitionValues) throws WorkerException;
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/6ec72de7/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/WarehousePartitionHelper.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/WarehousePartitionHelper.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/WarehousePartitionHelper.java
new file mode 100644
index 0000000..c2edee3
--- /dev/null
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/WarehousePartitionHelper.java
@@ -0,0 +1,69 @@
+package org.apache.hive.hcatalog.streaming.mutate.worker;
+
+import java.io.IOException;
+import java.util.LinkedHashMap;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.metastore.Warehouse;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+
+/**
+ * A {@link PartitionHelper} implementation that uses the {@link Warehouse} class to obtain partition path information.
+ * As this does not require a connection to the meta store database it is safe to use in workers that are distributed on
+ * a cluster. However, it does not support the creation of new partitions so you will need to provide a mechanism to
+ * collect affected partitions in your merge job and create them from your client.
+ */
+class WarehousePartitionHelper implements PartitionHelper {
+
+  private final Warehouse warehouse;
+  private final Path tablePath;
+  private final LinkedHashMap<String, String> partitions;
+  private final List<String> partitionColumns;
+
+  WarehousePartitionHelper(Configuration configuration, Path tablePath, List<String> partitionColumns)
+      throws MetaException {
+    this.tablePath = tablePath;
+    this.partitionColumns = partitionColumns;
+    this.partitions = new LinkedHashMap<>(partitionColumns.size());
+    for (String partitionColumn : partitionColumns) {
+      partitions.put(partitionColumn, null);
+    }
+    warehouse = new Warehouse(configuration);
+  }
+
+  @Override
+  public Path getPathForPartition(List<String> partitionValues) throws WorkerException {
+    if (partitionValues.size() != partitionColumns.size()) {
+      throw new IllegalArgumentException("Incorrect number of partition values. columns=" + partitionColumns
+          + ",values=" + partitionValues);
+    }
+    if (partitionColumns.isEmpty()) {
+      return tablePath;
+    }
+    for (int columnIndex = 0; columnIndex < partitionValues.size(); columnIndex++) {
+      String partitionColumn = partitionColumns.get(columnIndex);
+      String partitionValue = partitionValues.get(columnIndex);
+      partitions.put(partitionColumn, partitionValue);
+    }
+    try {
+      return warehouse.getPartitionPath(tablePath, partitions);
+    } catch (MetaException e) {
+      throw new WorkerException("Unable to determine partition path. tablePath=" + tablePath + ",partition="
+          + partitionValues, e);
+    }
+  }
+
+  /** Throws {@link UnsupportedOperationException}. */
+  @Override
+  public void createPartitionIfNotExists(List<String> newPartitionValues) throws WorkerException {
+    throw new UnsupportedOperationException("You require a connection to the meta store to do this.");
+  }
+
+  @Override
+  public void close() throws IOException {
+    // Nothing to close here.
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/6ec72de7/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestMetaStorePartitionHelper.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestMetaStorePartitionHelper.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestMetaStorePartitionHelper.java
new file mode 100644
index 0000000..cc4173e
--- /dev/null
+++ b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestMetaStorePartitionHelper.java
@@ -0,0 +1,112 @@
+package org.apache.hive.hcatalog.streaming.mutate.worker;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyZeroInteractions;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Captor;
+import org.mockito.Mock;
+import org.mockito.runners.MockitoJUnitRunner;
+
+@RunWith(MockitoJUnitRunner.class)
+public class TestMetaStorePartitionHelper {
+
+  private static final Path TABLE_PATH = new Path("table");
+  private static final String TABLE_LOCATION = TABLE_PATH.toString();
+
+  private static final FieldSchema PARTITION_KEY_A = new FieldSchema("A", "string", null);
+  private static final FieldSchema PARTITION_KEY_B = new FieldSchema("B", "string", null);
+  private static final List<FieldSchema> PARTITION_KEYS = Arrays.asList(PARTITION_KEY_A, PARTITION_KEY_B);
+  private static final Path PARTITION_PATH = new Path(TABLE_PATH, "a=1/b=2");
+  private static final String PARTITION_LOCATION = PARTITION_PATH.toString();
+
+  private static final String DATABASE_NAME = "db";
+  private static final String TABLE_NAME = "one";
+
+  private static final List<String> UNPARTITIONED_VALUES = Collections.emptyList();
+  private static final List<String> PARTITIONED_VALUES = Arrays.asList("1", "2");
+
+  @Mock
+  private IMetaStoreClient mockClient;
+  @Mock
+  private Table mockTable;
+  private StorageDescriptor tableStorageDescriptor = new StorageDescriptor();
+
+  @Mock
+  private Partition mockPartition;
+  @Mock
+  private StorageDescriptor mockPartitionStorageDescriptor;
+  @Captor
+  private ArgumentCaptor<Partition> partitionCaptor;
+
+  private PartitionHelper helper;
+
+  @Before
+  public void injectMocks() throws Exception {
+    when(mockClient.getTable(DATABASE_NAME, TABLE_NAME)).thenReturn(mockTable);
+    when(mockTable.getDbName()).thenReturn(DATABASE_NAME);
+    when(mockTable.getTableName()).thenReturn(TABLE_NAME);
+    when(mockTable.getPartitionKeys()).thenReturn(PARTITION_KEYS);
+    when(mockTable.getSd()).thenReturn(tableStorageDescriptor);
+    tableStorageDescriptor.setLocation(TABLE_LOCATION);
+
+    when(mockClient.getPartition(DATABASE_NAME, TABLE_NAME, PARTITIONED_VALUES)).thenReturn(mockPartition);
+    when(mockPartition.getSd()).thenReturn(mockPartitionStorageDescriptor);
+    when(mockPartitionStorageDescriptor.getLocation()).thenReturn(PARTITION_LOCATION);
+
+    helper = new MetaStorePartitionHelper(mockClient, DATABASE_NAME, TABLE_NAME, TABLE_PATH);
+  }
+
+  @Test
+  public void getPathForUnpartitionedTable() throws Exception {
+    Path path = helper.getPathForPartition(UNPARTITIONED_VALUES);
+    assertThat(path, is(TABLE_PATH));
+    verifyZeroInteractions(mockClient);
+  }
+
+  @Test
+  public void getPathForPartitionedTable() throws Exception {
+    Path path = helper.getPathForPartition(PARTITIONED_VALUES);
+    assertThat(path, is(PARTITION_PATH));
+  }
+
+  @Test
+  public void createOnUnpartitionTableDoesNothing() throws Exception {
+    helper.createPartitionIfNotExists(UNPARTITIONED_VALUES);
+    verifyZeroInteractions(mockClient);
+  }
+
+  @Test
+  public void createOnPartitionTable() throws Exception {
+    helper.createPartitionIfNotExists(PARTITIONED_VALUES);
+
+    verify(mockClient).add_partition(partitionCaptor.capture());
+    Partition actual = partitionCaptor.getValue();
+    assertThat(actual.getSd().getLocation(), is(PARTITION_LOCATION));
+    assertThat(actual.getValues(), is(PARTITIONED_VALUES));
+  }
+
+  @Test
+  public void closeSucceeds() throws IOException {
+    helper.close();
+    verify(mockClient).close();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/6ec72de7/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestMutatorCoordinator.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestMutatorCoordinator.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestMutatorCoordinator.java
index 6e9ffa2..2983d12 100644
--- a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestMutatorCoordinator.java
+++ b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestMutatorCoordinator.java
@@ -2,8 +2,10 @@ package org.apache.hive.hcatalog.streaming.mutate.worker;
 
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyInt;
+import static org.mockito.Matchers.anyList;
 import static org.mockito.Matchers.anyLong;
 import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.verifyZeroInteractions;
@@ -15,7 +17,6 @@ import java.util.List;
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.ql.io.RecordIdentifier;
 import org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat;
 import org.apache.hive.hcatalog.streaming.mutate.client.AcidTable;
@@ -42,11 +43,9 @@ public class TestMutatorCoordinator {
   private static final RecordIdentifier ROW__ID_INSERT = new RecordIdentifier(-1L, BUCKET_ID, -1L);
 
   @Mock
-  private IMetaStoreClient mockMetaStoreClient;
-  @Mock
   private MutatorFactory mockMutatorFactory;
   @Mock
-  private CreatePartitionHelper mockPartitionHelper;
+  private PartitionHelper mockPartitionHelper;
   @Mock
   private GroupingValidator mockGroupingValidator;
   @Mock
@@ -79,8 +78,8 @@ public class TestMutatorCoordinator {
     when(mockSequenceValidator.isInSequence(any(RecordIdentifier.class))).thenReturn(true);
     when(mockGroupingValidator.isInSequence(any(List.class), anyInt())).thenReturn(true);
 
-    coordinator = new MutatorCoordinator(mockMetaStoreClient, configuration, mockMutatorFactory, mockPartitionHelper,
-        mockGroupingValidator, mockSequenceValidator, mockAcidTable, false);
+    coordinator = new MutatorCoordinator(configuration, mockMutatorFactory, mockPartitionHelper, mockGroupingValidator,
+        mockSequenceValidator, mockAcidTable, false);
   }
 
   @Test
@@ -127,7 +126,6 @@ public class TestMutatorCoordinator {
     coordinator.update(UNPARTITIONED, RECORD);
     coordinator.delete(UNPARTITIONED, RECORD);
 
-    verify(mockPartitionHelper).createPartitionIfNotExists(UNPARTITIONED);
     verify(mockMutatorFactory).newMutator(any(OrcOutputFormat.class), eq(TRANSACTION_ID), eq(PATH_A), eq(BUCKET_ID));
     verify(mockMutatorFactory)
         .newMutator(any(OrcOutputFormat.class), eq(TRANSACTION_ID), eq(PATH_A), eq(BUCKET_ID + 1));
@@ -145,12 +143,11 @@ public class TestMutatorCoordinator {
     when(mockPartitionHelper.getPathForPartition(PARTITION_A)).thenReturn(PATH_A);
     when(mockPartitionHelper.getPathForPartition(PARTITION_B)).thenReturn(PATH_B);
 
-    coordinator.update(PARTITION_A, RECORD);
-    coordinator.delete(PARTITION_B, RECORD);
-    coordinator.update(PARTITION_B, RECORD);
-    coordinator.insert(PARTITION_B, RECORD);
+    coordinator.update(PARTITION_A, RECORD); /* PaB0 */
+    coordinator.insert(PARTITION_B, RECORD); /* PbB0 */
+    coordinator.delete(PARTITION_B, RECORD); /* PbB0 */
+    coordinator.update(PARTITION_B, RECORD); /* PbB1 */
 
-    verify(mockPartitionHelper).createPartitionIfNotExists(PARTITION_A);
     verify(mockPartitionHelper).createPartitionIfNotExists(PARTITION_B);
     verify(mockMutatorFactory).newMutator(any(OrcOutputFormat.class), eq(TRANSACTION_ID), eq(PATH_A), eq(BUCKET_ID));
     verify(mockMutatorFactory, times(2)).newMutator(any(OrcOutputFormat.class), eq(TRANSACTION_ID), eq(PATH_B),
@@ -163,6 +160,18 @@ public class TestMutatorCoordinator {
     verify(mockSequenceValidator, times(4)).reset();
   }
 
+  @Test
+  public void partitionThenBucketChangesNoCreateAsPartitionEstablished() throws Exception {
+    when(mockRecordInspector.extractRecordIdentifier(RECORD)).thenReturn(ROW__ID_B0_R0, ROW__ID_INSERT);
+    when(mockBucketIdResolver.computeBucketId(RECORD)).thenReturn(0, 0);
+    when(mockPartitionHelper.getPathForPartition(PARTITION_B)).thenReturn(PATH_B);
+
+    coordinator.delete(PARTITION_B, RECORD); /* PbB0 */
+    coordinator.insert(PARTITION_B, RECORD); /* PbB0 */
+
+    verify(mockPartitionHelper, never()).createPartitionIfNotExists(anyList());
+  }
+
   @Test(expected = RecordSequenceException.class)
   public void outOfSequence() throws Exception {
     when(mockSequenceValidator.isInSequence(any(RecordIdentifier.class))).thenReturn(false);
@@ -175,14 +184,14 @@ public class TestMutatorCoordinator {
     verify(mockMutator).update(RECORD);
     verify(mockMutator).delete(RECORD);
   }
-  
+
   @Test(expected = GroupRevisitedException.class)
   public void revisitGroup() throws Exception {
     when(mockGroupingValidator.isInSequence(any(List.class), anyInt())).thenReturn(false);
-    
+
     coordinator.update(UNPARTITIONED, RECORD);
     coordinator.delete(UNPARTITIONED, RECORD);
-    
+
     verify(mockPartitionHelper).createPartitionIfNotExists(UNPARTITIONED);
     verify(mockMutatorFactory).newMutator(any(OrcOutputFormat.class), eq(TRANSACTION_ID), eq(PATH_A), eq(BUCKET_ID));
     verify(mockMutator).update(RECORD);
@@ -230,5 +239,6 @@ public class TestMutatorCoordinator {
     coordinator.close();
 
     verify(mockMutator).close();
+    verify(mockPartitionHelper).close();
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/6ec72de7/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestWarehousePartitionHelper.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestWarehousePartitionHelper.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestWarehousePartitionHelper.java
new file mode 100644
index 0000000..e779771
--- /dev/null
+++ b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestWarehousePartitionHelper.java
@@ -0,0 +1,57 @@
+package org.apache.hive.hcatalog.streaming.mutate.worker;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.junit.Test;
+
+public class TestWarehousePartitionHelper {
+
+  private static final Configuration CONFIGURATION = new Configuration();
+  private static final Path TABLE_PATH = new Path("table");
+  
+  private static final List<String> UNPARTITIONED_COLUMNS = Collections.emptyList();
+  private static final List<String> UNPARTITIONED_VALUES = Collections.emptyList();
+  
+  private static final List<String> PARTITIONED_COLUMNS = Arrays.asList("A", "B");
+  private static final List<String> PARTITIONED_VALUES = Arrays.asList("1", "2");
+  
+  private final PartitionHelper unpartitionedHelper;
+  private final PartitionHelper partitionedHelper;
+
+  public TestWarehousePartitionHelper() throws Exception {
+    unpartitionedHelper = new WarehousePartitionHelper(CONFIGURATION, TABLE_PATH, UNPARTITIONED_COLUMNS);
+    partitionedHelper = new WarehousePartitionHelper(CONFIGURATION, TABLE_PATH, PARTITIONED_COLUMNS);
+  }
+
+  @Test(expected = UnsupportedOperationException.class)
+  public void createNotSupported() throws Exception {
+    unpartitionedHelper.createPartitionIfNotExists(UNPARTITIONED_VALUES);
+  }
+
+  @Test
+  public void getPathForUnpartitionedTable() throws Exception {
+    Path path = unpartitionedHelper.getPathForPartition(UNPARTITIONED_VALUES);
+    assertThat(path, is(TABLE_PATH));
+  }
+
+  @Test
+  public void getPathForPartitionedTable() throws Exception {
+    Path path = partitionedHelper.getPathForPartition(PARTITIONED_VALUES);
+    assertThat(path, is(new Path(TABLE_PATH, "A=1/B=2")));
+  }
+
+  @Test
+  public void closeSucceeds() throws IOException {
+    partitionedHelper.close();
+    unpartitionedHelper.close();
+  }
+  
+}


[41/43] hive git commit: HIVE-11257: CBO: Calcite Operator To Hive Operator (Calcite Return Path): Method isCombinablePredicate in HiveJoinToMultiJoinRule should be extended to support MultiJoin operators merge (Jesus Camacho Rodriguez, reviewed by Pengc

Posted by xu...@apache.org.
HIVE-11257: CBO: Calcite Operator To Hive Operator (Calcite Return Path): Method isCombinablePredicate in HiveJoinToMultiJoinRule should be extended to support MultiJoin operators merge (Jesus Camacho Rodriguez, reviewed by Pengcheng Xiong)


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

Branch: refs/heads/spark
Commit: bfe859169ace54c050a88a8424a209fd4e735aa0
Parents: 0ec5185
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Wed Jul 29 21:05:18 2015 +0200
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Thu Jul 30 12:35:27 2015 +0200

----------------------------------------------------------------------
 .../calcite/rules/HiveJoinToMultiJoinRule.java  | 82 ++++++++++++++------
 1 file changed, 58 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/bfe85916/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinToMultiJoinRule.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinToMultiJoinRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinToMultiJoinRule.java
index d0a29a7..35dbda9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinToMultiJoinRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinToMultiJoinRule.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hive.ql.optimizer.calcite.rules;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Set;
 
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
@@ -95,13 +96,13 @@ public class HiveJoinToMultiJoinRule extends RelOptRule {
     //    The result of the swapping operation is either
     //    i)  a Project or,
     //    ii) if the project is trivial, a raw join
-    final Join newJoin;
+    final HiveJoin newJoin;
     Project topProject = null;
-    if (swapped instanceof Join) {
-      newJoin = (Join) swapped;
+    if (swapped instanceof HiveJoin) {
+      newJoin = (HiveJoin) swapped;
     } else {
       topProject = (Project) swapped;
-      newJoin = (Join) swapped.getInput(0);
+      newJoin = (HiveJoin) swapped.getInput(0);
     }
 
     // 3. We try to merge the join with the right child
@@ -119,7 +120,7 @@ public class HiveJoinToMultiJoinRule extends RelOptRule {
 
   // This method tries to merge the join with its left child. The left
   // child should be a join for this to happen.
-  private static RelNode mergeJoin(Join join, RelNode left, RelNode right) {
+  private static RelNode mergeJoin(HiveJoin join, RelNode left, RelNode right) {
     final RexBuilder rexBuilder = join.getCluster().getRexBuilder();
 
     // We check whether the join can be combined with any of its children
@@ -135,28 +136,33 @@ public class HiveJoinToMultiJoinRule extends RelOptRule {
       final List<Pair<Integer,Integer>> leftJoinInputs;
       final List<JoinRelType> leftJoinTypes;
       final List<RexNode> leftJoinFilters;
+      boolean combinable;
       if (left instanceof HiveJoin) {
         HiveJoin hj = (HiveJoin) left;
         leftCondition = hj.getCondition();
         leftJoinInputs = ImmutableList.of(Pair.of(0, 1));
         leftJoinTypes = ImmutableList.of(hj.getJoinType());
         leftJoinFilters = ImmutableList.of(hj.getJoinFilter());
+        try {
+          combinable = isCombinableJoin(join, hj);
+        } catch (CalciteSemanticException e) {
+          LOG.trace("Failed to merge join-join", e);
+          combinable = false;
+        }
       } else {
         HiveMultiJoin hmj = (HiveMultiJoin) left;
         leftCondition = hmj.getCondition();
         leftJoinInputs = hmj.getJoinInputs();
         leftJoinTypes = hmj.getJoinTypes();
         leftJoinFilters = hmj.getJoinFilters();
+        try {
+          combinable = isCombinableJoin(join, hmj);
+        } catch (CalciteSemanticException e) {
+          LOG.trace("Failed to merge join-multijoin", e);
+          combinable = false;
+        }
       }
 
-      boolean combinable;
-      try {
-        combinable = isCombinablePredicate(join, join.getCondition(),
-                leftCondition);
-      } catch (CalciteSemanticException e) {
-        LOG.trace("Failed to merge joins", e);
-        combinable = false;
-      }
       if (combinable) {
         newJoinCondition.add(leftCondition);
         for (int i = 0; i < leftJoinInputs.size(); i++) {
@@ -243,20 +249,48 @@ public class HiveJoinToMultiJoinRule extends RelOptRule {
             joinFilters);
   }
 
-  private static boolean isCombinablePredicate(Join join,
-          RexNode condition, RexNode otherCondition) throws CalciteSemanticException {
+  /*
+   * Returns true if the join conditions execute over the same keys
+   */
+  private static boolean isCombinableJoin(HiveJoin join, HiveJoin leftChildJoin)
+          throws CalciteSemanticException {
+    final JoinPredicateInfo joinPredInfo = HiveCalciteUtil.JoinPredicateInfo.
+            constructJoinPredicateInfo(join, join.getCondition());
+    final JoinPredicateInfo leftChildJoinPredInfo = HiveCalciteUtil.JoinPredicateInfo.
+            constructJoinPredicateInfo(leftChildJoin, leftChildJoin.getCondition());
+    return isCombinablePredicate(joinPredInfo, leftChildJoinPredInfo, leftChildJoin.getInputs().size());
+  }
+
+  /*
+   * Returns true if the join conditions execute over the same keys
+   */
+  private static boolean isCombinableJoin(HiveJoin join, HiveMultiJoin leftChildJoin)
+          throws CalciteSemanticException {
     final JoinPredicateInfo joinPredInfo = HiveCalciteUtil.JoinPredicateInfo.
-            constructJoinPredicateInfo(join, condition);
-    final JoinPredicateInfo otherJoinPredInfo = HiveCalciteUtil.JoinPredicateInfo.
-            constructJoinPredicateInfo(join, otherCondition);
-    if (joinPredInfo.getProjsFromLeftPartOfJoinKeysInJoinSchema().
-            equals(otherJoinPredInfo.getProjsFromLeftPartOfJoinKeysInJoinSchema())) {
+            constructJoinPredicateInfo(join, join.getCondition());
+    final JoinPredicateInfo leftChildJoinPredInfo = HiveCalciteUtil.JoinPredicateInfo.
+            constructJoinPredicateInfo(leftChildJoin, leftChildJoin.getCondition());
+    return isCombinablePredicate(joinPredInfo, leftChildJoinPredInfo, leftChildJoin.getInputs().size());
+  }
+
+  /*
+   * To be able to combine a parent join and its left input join child,
+   * the left keys over which the parent join is executed need to be the same
+   * than those of the child join.
+   * Thus, we iterate over the different inputs of the child, checking if the
+   * keys of the parent are the same
+   */
+  private static boolean isCombinablePredicate(JoinPredicateInfo joinPredInfo,
+          JoinPredicateInfo leftChildJoinPredInfo, int noLeftChildInputs) throws CalciteSemanticException {
+    Set<Integer> keys = joinPredInfo.getProjsJoinKeysInChildSchema(0);
+    if (keys.isEmpty()) {
       return false;
     }
-    if (joinPredInfo.getProjsFromRightPartOfJoinKeysInJoinSchema().
-            equals(otherJoinPredInfo.getProjsFromRightPartOfJoinKeysInJoinSchema())) {
-      return false;
+    for (int i = 0; i < noLeftChildInputs; i++) {
+      if (keys.equals(leftChildJoinPredInfo.getProjsJoinKeysInJoinSchema(i))) {
+        return true;
+      }
     }
-    return true;
+    return false;
   }
 }


[21/43] hive git commit: HIVE-11344 : HIVE-9845 makes HCatSplit.write modify the split so that PartInfo objects are unusable after it (Sushanth Sowmyan, reviewed by Mithun Radhakrishnan)

Posted by xu...@apache.org.
HIVE-11344 : HIVE-9845 makes HCatSplit.write modify the split so that PartInfo objects are unusable after it (Sushanth Sowmyan, reviewed by Mithun Radhakrishnan)


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

Branch: refs/heads/spark
Commit: 9f474263b5c47dc25c096a0b54794ebcda146bed
Parents: 92e9885
Author: Sushanth Sowmyan <kh...@gmail.com>
Authored: Mon Jul 27 09:21:54 2015 -0700
Committer: Sushanth Sowmyan <kh...@gmail.com>
Committed: Mon Jul 27 09:29:18 2015 -0700

----------------------------------------------------------------------
 .../hive/hcatalog/mapreduce/PartInfo.java       | 32 +++++++++++++++-----
 1 file changed, 25 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/9f474263/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/PartInfo.java
----------------------------------------------------------------------
diff --git a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/PartInfo.java b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/PartInfo.java
index fca0a92..5b88505 100644
--- a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/PartInfo.java
+++ b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/PartInfo.java
@@ -173,7 +173,16 @@ public class PartInfo implements Serializable {
 
   void setTableInfo(HCatTableInfo thatTableInfo) {
     this.tableInfo = thatTableInfo;
+    restoreLocalInfoFromTableInfo();
+  }
 
+  /**
+   * Undoes the effects of compression( dedupWithTableInfo() ) during serialization,
+   * and restores PartInfo fields to return original data.
+   * Can be called idempotently, repeatably.
+   */
+  private void restoreLocalInfoFromTableInfo() {
+    assert tableInfo != null : "TableInfo can't be null at this point.";
     if (partitionSchema == null) {
       partitionSchema = tableInfo.getDataColumns();
     }
@@ -196,15 +205,10 @@ public class PartInfo implements Serializable {
   }
 
   /**
-   * Serialization method. Suppresses serialization of redundant information that's already
-   * available from TableInfo.
+   * Finds commonalities with TableInfo, and suppresses (nulls) fields if they are identical
    */
-  private void writeObject(ObjectOutputStream oos)
-      throws IOException {
-    // Suppress commonality with TableInfo.
-
+  private void dedupWithTableInfo() {
     assert tableInfo != null : "TableInfo can't be null at this point.";
-
     if (partitionSchema != null) {
       if (partitionSchema.equals(tableInfo.getDataColumns())) {
         partitionSchema = null;
@@ -260,7 +264,21 @@ public class PartInfo implements Serializable {
         }
       }
     }
+  }
 
+  /**
+   * Serialization method used by java serialization.
+   * Suppresses serialization of redundant information that's already available from
+   * TableInfo before writing out, so as to minimize amount of serialized space but
+   * restore it back before returning, so that PartInfo object is still usable afterwards
+   * (See HIVE-8485 and HIVE-11344 for details.)
+   */
+  private void writeObject(ObjectOutputStream oos)
+      throws IOException {
+    dedupWithTableInfo();
     oos.defaultWriteObject();
+    restoreLocalInfoFromTableInfo();
   }
+
+
 }


[23/43] hive git commit: HIVE-11196 : Utilities.getPartitionDesc() should try to reuse TableDesc object (Hari Subramaniyan, reviewd by Jesus Camacho Rodriguez)

Posted by xu...@apache.org.
HIVE-11196 : Utilities.getPartitionDesc() should try to reuse TableDesc object (Hari Subramaniyan, reviewd 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/8dcead4e
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/8dcead4e
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/8dcead4e

Branch: refs/heads/spark
Commit: 8dcead4ea1c1c6bb08eace1d8969694f8bff6ac7
Parents: c2dc452
Author: Hari Subramaniyan <ha...@apache.org>
Authored: Mon Jul 27 13:46:19 2015 -0700
Committer: Hari Subramaniyan <ha...@apache.org>
Committed: Mon Jul 27 13:46:19 2015 -0700

----------------------------------------------------------------------
 .../mapreduce/TestHCatMultiOutputFormat.java    |  6 ++-
 .../apache/hadoop/hive/ql/exec/Utilities.java   |  9 +++--
 .../hive/ql/optimizer/GenMapRedUtils.java       | 18 ++++++---
 .../hive/ql/optimizer/SimpleFetchOptimizer.java |  5 ++-
 .../hadoop/hive/ql/plan/PartitionDesc.java      | 39 ++++++++++++++++----
 5 files changed, 56 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/8dcead4e/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatMultiOutputFormat.java
----------------------------------------------------------------------
diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatMultiOutputFormat.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatMultiOutputFormat.java
index 049de54..8148faa 100644
--- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatMultiOutputFormat.java
+++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatMultiOutputFormat.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.plan.FetchWork;
 import org.apache.hadoop.hive.ql.plan.PartitionDesc;
+import org.apache.hadoop.hive.ql.plan.TableDesc;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.LongWritable;
@@ -379,11 +380,12 @@ public class TestHCatMultiOutputFormat {
       List<Partition> partitions = hive.getPartitions(tbl);
       List<PartitionDesc> partDesc = new ArrayList<PartitionDesc>();
       List<Path> partLocs = new ArrayList<Path>();
+      TableDesc tableDesc = Utilities.getTableDesc(tbl);
       for (Partition part : partitions) {
         partLocs.add(part.getDataLocation());
-        partDesc.add(Utilities.getPartitionDesc(part));
+        partDesc.add(Utilities.getPartitionDescFromTableDesc(tableDesc, part, true));
       }
-      work = new FetchWork(partLocs, partDesc, Utilities.getTableDesc(tbl));
+      work = new FetchWork(partLocs, partDesc, tableDesc);
       work.setLimit(100);
     } else {
       work = new FetchWork(tbl.getDataLocation(), Utilities.getTableDesc(tbl));

http://git-wip-us.apache.org/repos/asf/hive/blob/8dcead4e/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
index d8e463d..d578f11 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.hive.ql.exec;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import java.beans.DefaultPersistenceDelegate;
 import java.beans.Encoder;
 import java.beans.ExceptionListener;
@@ -80,6 +82,7 @@ import java.util.zip.DeflaterOutputStream;
 import java.util.zip.InflaterInputStream;
 
 import org.antlr.runtime.CommonToken;
+import org.apache.calcite.util.ChunkList;
 import org.apache.commons.codec.binary.Base64;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang.WordUtils;
@@ -1230,9 +1233,9 @@ public final class Utilities {
     return (new PartitionDesc(part));
   }
 
-  public static PartitionDesc getPartitionDescFromTableDesc(TableDesc tblDesc, Partition part)
-      throws HiveException {
-    return new PartitionDesc(part, tblDesc);
+  public static PartitionDesc getPartitionDescFromTableDesc(TableDesc tblDesc, Partition part,
+    boolean usePartSchemaProperties) throws HiveException {
+    return new PartitionDesc(part, tblDesc, usePartSchemaProperties);
   }
 
   private static String getOpTreeSkel_helper(Operator<?> op, String indent) {

http://git-wip-us.apache.org/repos/asf/hive/blob/8dcead4e/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java
index 29854d8..693d8c7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java
@@ -487,8 +487,6 @@ public final class GenMapRedUtils {
     ArrayList<PartitionDesc> partDesc = new ArrayList<PartitionDesc>();
 
     Path tblDir = null;
-    TableDesc tblDesc = null;
-
     plan.setNameToSplitSample(parseCtx.getNameToSplitSample());
 
     if (partsList == null) {
@@ -575,6 +573,8 @@ public final class GenMapRedUtils {
     //This read entity is a direct read entity and not an indirect read (that is when
     // this is being read because it is a dependency of a view).
     boolean isDirectRead = (parentViewInfo == null);
+    TableDesc tblDesc = null;
+    boolean initTableDesc = false;
 
     for (Partition part : parts) {
       if (part.getTable().isPartitioned()) {
@@ -647,12 +647,18 @@ public final class GenMapRedUtils {
 
       // is it a partitioned table ?
       if (!part.getTable().isPartitioned()) {
-        assert ((tblDir == null) && (tblDesc == null));
+        assert (tblDir == null);
 
         tblDir = paths[0];
-        tblDesc = Utilities.getTableDesc(part.getTable());
+        if (!initTableDesc) {
+          tblDesc = Utilities.getTableDesc(part.getTable());
+          initTableDesc = true;
+        }
       } else if (tblDesc == null) {
-        tblDesc = Utilities.getTableDesc(part.getTable());
+        if (!initTableDesc) {
+          tblDesc = Utilities.getTableDesc(part.getTable());
+          initTableDesc = true;
+        }
       }
 
       if (props != null) {
@@ -678,7 +684,7 @@ public final class GenMapRedUtils {
             partDesc.add(Utilities.getPartitionDesc(part));
           }
           else {
-            partDesc.add(Utilities.getPartitionDescFromTableDesc(tblDesc, part));
+            partDesc.add(Utilities.getPartitionDescFromTableDesc(tblDesc, part, false));
           }
         } catch (HiveException e) {
           LOG.error(org.apache.hadoop.util.StringUtils.stringifyException(e));

http://git-wip-us.apache.org/repos/asf/hive/blob/8dcead4e/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchOptimizer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchOptimizer.java
index 317454d..3859177 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchOptimizer.java
@@ -369,9 +369,10 @@ public class SimpleFetchOptimizer implements Transform {
 
     private FetchWork convertToWork() throws HiveException {
       inputs.clear();
+      TableDesc tableDesc = Utilities.getTableDesc(table);
       if (!table.isPartitioned()) {
         inputs.add(new ReadEntity(table, parent, !table.isView() && parent == null));
-        FetchWork work = new FetchWork(table.getPath(), Utilities.getTableDesc(table));
+        FetchWork work = new FetchWork(table.getPath(), tableDesc);
         PlanUtils.configureInputJobPropertiesForStorageHandler(work.getTblDesc());
         work.setSplitSample(splitSample);
         return work;
@@ -382,7 +383,7 @@ public class SimpleFetchOptimizer implements Transform {
       for (Partition partition : partsList.getNotDeniedPartns()) {
         inputs.add(new ReadEntity(partition, parent, parent == null));
         listP.add(partition.getDataLocation());
-        partP.add(Utilities.getPartitionDesc(partition));
+        partP.add(Utilities.getPartitionDescFromTableDesc(tableDesc, partition, true));
       }
       Table sourceTable = partsList.getSourceTable();
       inputs.add(new ReadEntity(sourceTable, parent, parent == null));

http://git-wip-us.apache.org/repos/asf/hive/blob/8dcead4e/ql/src/java/org/apache/hadoop/hive/ql/plan/PartitionDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/PartitionDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/PartitionDesc.java
index b123511..864301c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/PartitionDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/PartitionDesc.java
@@ -81,18 +81,41 @@ public class PartitionDesc implements Serializable, Cloneable {
   }
 
   public PartitionDesc(final Partition part) throws HiveException {
-    this.tableDesc = Utilities.getTableDesc(part.getTable());
+    PartitionDescConstructorHelper(part, Utilities.getTableDesc(part.getTable()), true);
     setProperties(part.getMetadataFromPartitionSchema());
-    partSpec = part.getSpec();
-    setInputFileFormatClass(part.getInputFormatClass());
-    setOutputFileFormatClass(part.getOutputFormatClass());
   }
 
-  public PartitionDesc(final Partition part,final TableDesc tblDesc) throws HiveException {
+  /** 
+   * @param part Partition
+   * @param tblDesc Table Descriptor
+   * @param usePartSchemaProperties Use Partition Schema Properties to set the
+   * partition descriptor properties. This is usually set to true by the caller
+   * if the table is partitioned, i.e. if the table has partition columns.
+   * @throws HiveException
+   */
+  public PartitionDesc(final Partition part,final TableDesc tblDesc,
+    boolean usePartSchemaProperties)
+    throws HiveException {
+    PartitionDescConstructorHelper(part,tblDesc, usePartSchemaProperties);
+    //We use partition schema properties to set the partition descriptor properties
+    // if usePartSchemaProperties is set to true.
+    if (usePartSchemaProperties) {
+      setProperties(part.getMetadataFromPartitionSchema());
+    } else {
+      // each partition maintains a large properties
+      setProperties(part.getSchemaFromTableSchema(tblDesc.getProperties()));
+    }
+  }
+
+  private void PartitionDescConstructorHelper(final Partition part,final TableDesc tblDesc, boolean setInputFileFormat)
+    throws HiveException {
     this.tableDesc = tblDesc;
-    setProperties(part.getSchemaFromTableSchema(tblDesc.getProperties())); // each partition maintains a large properties
-    partSpec = part.getSpec();
-    setOutputFileFormatClass(part.getInputFormatClass());
+    this.partSpec = part.getSpec();
+    if (setInputFileFormat) {
+      setInputFileFormatClass(part.getInputFormatClass());
+    } else {
+      setOutputFileFormatClass(part.getInputFormatClass());
+    }
     setOutputFileFormatClass(part.getOutputFormatClass());
   }
 


[26/43] hive git commit: HIVE-10799. Refactor the SearchArgumentFactory to remove the AST-specific factory. (omalley reviewed by prasanth_j)

Posted by xu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/c178a6e9/serde/src/java/org/apache/hadoop/hive/ql/io/sarg/PredicateLeaf.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/ql/io/sarg/PredicateLeaf.java b/serde/src/java/org/apache/hadoop/hive/ql/io/sarg/PredicateLeaf.java
index 0a95363..3a92565 100644
--- a/serde/src/java/org/apache/hadoop/hive/ql/io/sarg/PredicateLeaf.java
+++ b/serde/src/java/org/apache/hadoop/hive/ql/io/sarg/PredicateLeaf.java
@@ -18,6 +18,10 @@
 
 package org.apache.hadoop.hive.ql.io.sarg;
 
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+
+import java.sql.Date;
+import java.sql.Timestamp;
 import java.util.List;
 
 /**
@@ -43,14 +47,27 @@ public interface PredicateLeaf {
    * The possible types for sargs.
    */
   public static enum Type {
-    INTEGER, // all of the integer types except long
-    LONG,
-    FLOAT,   // float and double
-    STRING,  // string, char, varchar
-    DATE,
-    DECIMAL,
-    TIMESTAMP,
-    BOOLEAN
+    INTEGER(Integer.class), // all of the integer types except long
+    LONG(Long.class),
+    FLOAT(Double.class),   // float and double
+    STRING(String.class),  // string, char, varchar
+    DATE(Date.class),
+    DECIMAL(HiveDecimalWritable.class),
+    TIMESTAMP(Timestamp.class),
+    BOOLEAN(Boolean.class);
+
+    private final Class cls;
+    Type(Class cls) {
+      this.cls = cls;
+    }
+
+    /**
+     * For all SARG leaves, the values must be the matching class.
+     * @return the value class
+     */
+    public Class getValueClass() {
+      return cls;
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/c178a6e9/serde/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgument.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgument.java b/serde/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgument.java
index 84604cb..bc0d503 100644
--- a/serde/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgument.java
+++ b/serde/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgument.java
@@ -215,58 +215,78 @@ public interface SearchArgument {
     /**
      * Add a less than leaf to the current item on the stack.
      * @param column the name of the column
+     * @param type the type of the expression
      * @param literal the literal
      * @return this
      */
-    public Builder lessThan(String column, Object literal);
+    public Builder lessThan(String column, PredicateLeaf.Type type,
+                            Object literal);
 
     /**
      * Add a less than equals leaf to the current item on the stack.
      * @param column the name of the column
+     * @param type the type of the expression
      * @param literal the literal
      * @return this
      */
-    public Builder lessThanEquals(String column, Object literal);
+    public Builder lessThanEquals(String column, PredicateLeaf.Type type,
+                                  Object literal);
 
     /**
      * Add an equals leaf to the current item on the stack.
      * @param column the name of the column
+     * @param type the type of the expression
      * @param literal the literal
      * @return this
      */
-    public Builder equals(String column, Object literal);
+    public Builder equals(String column, PredicateLeaf.Type type,
+                          Object literal);
 
     /**
      * Add a null safe equals leaf to the current item on the stack.
      * @param column the name of the column
+     * @param type the type of the expression
      * @param literal the literal
      * @return this
      */
-    public Builder nullSafeEquals(String column, Object literal);
+    public Builder nullSafeEquals(String column, PredicateLeaf.Type type,
+                                  Object literal);
 
     /**
      * Add an in leaf to the current item on the stack.
      * @param column the name of the column
+     * @param type the type of the expression
      * @param literal the literal
      * @return this
      */
-    public Builder in(String column, Object... literal);
+    public Builder in(String column, PredicateLeaf.Type type,
+                      Object... literal);
 
     /**
      * Add an is null leaf to the current item on the stack.
      * @param column the name of the column
+     * @param type the type of the expression
      * @return this
      */
-    public Builder isNull(String column);
+    public Builder isNull(String column, PredicateLeaf.Type type);
 
     /**
      * Add a between leaf to the current item on the stack.
      * @param column the name of the column
+     * @param type the type of the expression
      * @param lower the literal
      * @param upper the literal
      * @return this
      */
-    public Builder between(String column, Object lower, Object upper);
+    public Builder between(String column, PredicateLeaf.Type type,
+                           Object lower, Object upper);
+
+    /**
+     * Add a truth value to the expression.
+     * @param truth
+     * @return this
+     */
+    public Builder literal(TruthValue truth);
 
     /**
      * Build and return the SearchArgument that has been defined. All of the

http://git-wip-us.apache.org/repos/asf/hive/blob/c178a6e9/serde/src/java/org/apache/hadoop/hive/serde2/io/HiveDecimalWritable.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/io/HiveDecimalWritable.java b/serde/src/java/org/apache/hadoop/hive/serde2/io/HiveDecimalWritable.java
index a17d2cc..885828a 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/io/HiveDecimalWritable.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/io/HiveDecimalWritable.java
@@ -46,6 +46,10 @@ public class HiveDecimalWritable implements WritableComparable<HiveDecimalWritab
   public HiveDecimalWritable() {
   }
 
+  public HiveDecimalWritable(String value) {
+    set(HiveDecimal.create(value));
+  }
+
   public HiveDecimalWritable(byte[] bytes, int scale) {
     set(bytes, scale);
   }
@@ -58,6 +62,10 @@ public class HiveDecimalWritable implements WritableComparable<HiveDecimalWritab
     set(value);
   }
 
+  public HiveDecimalWritable(long value) {
+    set((HiveDecimal.create(value)));
+  }
+
   public void set(HiveDecimal value) {
     set(value.unscaledValue().toByteArray(), value.scale());
   }


[33/43] hive git commit: HIVE-11321. Move OrcFile.OrcTableProperties from OrcFile into OrcConf. (omalley reviewed by prasanthj)

Posted by xu...@apache.org.
HIVE-11321. Move OrcFile.OrcTableProperties from OrcFile into OrcConf.
(omalley reviewed by prasanthj)


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

Branch: refs/heads/spark
Commit: cd2b49970cec15c4a4b281301e28f71f25deba4d
Parents: 0ead9fe
Author: Owen O'Malley <om...@apache.org>
Authored: Tue Jul 28 13:02:07 2015 -0700
Committer: Owen O'Malley <om...@apache.org>
Committed: Tue Jul 28 13:02:07 2015 -0700

----------------------------------------------------------------------
 .../hive/hcatalog/mapreduce/SpecialCases.java   |   8 +-
 .../apache/hadoop/hive/ql/io/orc/OrcConf.java   | 129 ++++++++++++-----
 .../apache/hadoop/hive/ql/io/orc/OrcFile.java   | 138 ++++++++-----------
 .../hadoop/hive/ql/io/orc/OrcOutputFormat.java  |  67 +--------
 4 files changed, 153 insertions(+), 189 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/cd2b4997/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/SpecialCases.java
----------------------------------------------------------------------
diff --git a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/SpecialCases.java b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/SpecialCases.java
index f38d53b..7da2ab0 100644
--- a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/SpecialCases.java
+++ b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/SpecialCases.java
@@ -24,7 +24,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.io.RCFileOutputFormat;
 import org.apache.hadoop.hive.ql.io.avro.AvroContainerOutputFormat;
-import org.apache.hadoop.hive.ql.io.orc.OrcFile;
+import org.apache.hadoop.hive.ql.io.orc.OrcConf;
 import org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat;
 import org.apache.hadoop.hive.serde2.avro.AvroSerDe;
 import org.apache.hadoop.hive.serde2.avro.AvroSerdeUtils;
@@ -85,10 +85,10 @@ public class SpecialCases {
       // them to job properties, so that it will be available in jobconf at runtime
       // See HIVE-5504 for details
       Map<String, String> tableProps = jobInfo.getTableInfo().getTable().getParameters();
-      for (OrcFile.OrcTableProperties property : OrcFile.OrcTableProperties.values()){
-        String propName = property.getPropName();
+      for (OrcConf property : OrcConf.values()){
+        String propName = property.getAttribute();
         if (tableProps.containsKey(propName)){
-          jobProperties.put(propName,tableProps.get(propName));
+          jobProperties.put(propName, tableProps.get(propName));
         }
       }
     } else if (ofclass == AvroContainerOutputFormat.class) {

http://git-wip-us.apache.org/repos/asf/hive/blob/cd2b4997/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcConf.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcConf.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcConf.java
index aeb0ec1..81b822f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcConf.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcConf.java
@@ -20,41 +20,48 @@ package org.apache.hadoop.hive.ql.io.orc;
 
 import org.apache.hadoop.conf.Configuration;
 
+import java.util.Properties;
+
 /**
  * Define the configuration properties that Orc understands.
  */
 public enum OrcConf {
-  STRIPE_SIZE("hive.exec.orc.default.stripe.size",
+  STRIPE_SIZE("orc.stripe.size", "hive.exec.orc.default.stripe.size",
       64L * 1024 * 1024,
       "Define the default ORC stripe size, in bytes."),
-  BLOCK_SIZE("hive.exec.orc.default.block.size", 256L * 1024 * 1024,
+  BLOCK_SIZE("orc.block.size", "hive.exec.orc.default.block.size",
+      256L * 1024 * 1024,
       "Define the default file system block size for ORC files."),
-  ROW_INDEX_STRIDE("hive.exec.orc.default.row.index.stride", 10000,
+  ENABLE_INDEXES("orc.create.index", "orc.create.index", true,
+      "Should the ORC writer create indexes as part of the file."),
+  ROW_INDEX_STRIDE("orc.row.index.stride",
+      "hive.exec.orc.default.row.index.stride", 10000,
       "Define the default ORC index stride in number of rows. (Stride is the\n"+
           " number of rows n index entry represents.)"),
-  BUFFER_SIZE("hive.exec.orc.default.buffer.size", 256 * 1024,
-      "Define the default ORC buffer size, in bytes."),
-  BLOCK_PADDING("hive.exec.orc.default.block.padding", true,
-      "Define the default block padding, which pads stripes to the HDFS\n" +
-          " block boundaries."),
-  COMPRESS("hive.exec.orc.default.compress", "ZLIB",
+  BUFFER_SIZE("orc.compress.size", "hive.exec.orc.default.buffer.size",
+      256 * 1024, "Define the default ORC buffer size, in bytes."),
+  BLOCK_PADDING("orc.block.padding", "hive.exec.orc.default.block.padding",
+      true,
+      "Define whether stripes should be padded to the HDFS block boundaries."),
+  COMPRESS("orc.compress", "hive.exec.orc.default.compress", "ZLIB",
       "Define the default compression codec for ORC file"),
-  WRITE_FORMAT("hive.exec.orc.write.format", null,
+  WRITE_FORMAT("orc.write.format", "hive.exec.orc.write.format", "0.12",
       "Define the version of the file to write. Possible values are 0.11 and\n"+
           " 0.12. If this parameter is not defined, ORC will use the run\n" +
-          " length encoding (RLE) introduced in Hive 0.12. Any value other\n" +
-          " than 0.11 results in the 0.12 encoding."),
-  ENCODING_STRATEGY("hive.exec.orc.encoding.strategy", "SPEED",
+          " length encoding (RLE) introduced in Hive 0.12."),
+  ENCODING_STRATEGY("orc.encoding.strategy", "hive.exec.orc.encoding.strategy",
+      "SPEED",
       "Define the encoding strategy to use while writing data. Changing this\n"+
           "will only affect the light weight encoding for integers. This\n" +
           "flag will not change the compression level of higher level\n" +
           "compression codec (like ZLIB)."),
-  COMPRESSION_STRATEGY("hive.exec.orc.compression.strategy", "SPEED",
+  COMPRESSION_STRATEGY("orc.compression.strategy",
+      "hive.exec.orc.compression.strategy", "SPEED",
       "Define the compression strategy to use while writing data.\n" +
           "This changes the compression level of higher level compression\n" +
           "codec (like ZLIB)."),
-  BLOCK_PADDING_TOLERANCE("hive.exec.orc.block.padding.tolerance",
-      0.05,
+  BLOCK_PADDING_TOLERANCE("orc.block.padding.tolerance",
+      "hive.exec.orc.block.padding.tolerance", 0.05,
       "Define the tolerance for block padding as a decimal fraction of\n" +
           "stripe size (for example, the default value 0.05 is 5% of the\n" +
           "stripe size). For the defaults of 64Mb ORC stripe and 256Mb HDFS\n" +
@@ -64,35 +71,45 @@ public enum OrcConf {
           "3.2Mb, a new smaller stripe will be inserted to fit within that\n" +
           "space. This will make sure that no stripe written will block\n" +
           " boundaries and cause remote reads within a node local task."),
-  BLOOM_FILTER_FPP("orc.default.bloom.fpp", 0.05,
+  BLOOM_FILTER_FPP("orc.bloom.filter.fpp", "orc.default.bloom.fpp", 0.05,
       "Define the default false positive probability for bloom filters."),
-  USE_ZEROCOPY("hive.exec.orc.zerocopy", false,
+  USE_ZEROCOPY("orc.use.zerocopy", "hive.exec.orc.zerocopy", false,
       "Use zerocopy reads with ORC. (This requires Hadoop 2.3 or later.)"),
-  SKIP_CORRUPT_DATA("hive.exec.orc.skip.corrupt.data", false,
+  SKIP_CORRUPT_DATA("orc.skip.corrupt.data", "hive.exec.orc.skip.corrupt.data",
+      false,
       "If ORC reader encounters corrupt data, this value will be used to\n" +
           "determine whether to skip the corrupt data or throw exception.\n" +
           "The default behavior is to throw exception."),
-  MEMORY_POOL("hive.exec.orc.memory.pool", 0.5,
+  MEMORY_POOL("orc.memory.pool", "hive.exec.orc.memory.pool", 0.5,
       "Maximum fraction of heap that can be used by ORC file writers"),
-  DICTIONARY_KEY_SIZE_THRESHOLD("hive.exec.orc.dictionary.key.size.threshold",
+  DICTIONARY_KEY_SIZE_THRESHOLD("orc.dictionary.key.threshold",
+      "hive.exec.orc.dictionary.key.size.threshold",
       0.8,
-      "If the number of keys in a dictionary is greater than this fraction\n" +
-          "of the total number of non-null rows, turn off dictionary\n" +
-          "encoding.  Use 1 to always use dictionary encoding."),
-  ROW_INDEX_STRIDE_DICTIONARY_CHECK("hive.orc.row.index.stride.dictionary.check",
+      "If the number of distinct keys in a dictionary is greater than this\n" +
+          "fraction of the total number of non-null rows, turn off \n" +
+          "dictionary encoding.  Use 1 to always use dictionary encoding."),
+  ROW_INDEX_STRIDE_DICTIONARY_CHECK("orc.dictionary.early.check",
+      "hive.orc.row.index.stride.dictionary.check",
       true,
       "If enabled dictionary check will happen after first row index stride\n" +
           "(default 10000 rows) else dictionary check will happen before\n" +
           "writing first stripe. In both cases, the decision to use\n" +
           "dictionary or not will be retained thereafter."),
+  BLOOM_FILTER_COLUMNS("orc.bloom.filter.columns", "orc.bloom.filter.columns",
+      "", "List of columns to create bloom filters for when writing.")
   ;
 
   private final String attribute;
+  private final String hiveConfName;
   private final Object defaultValue;
   private final String description;
 
-  OrcConf(String attribute, Object defaultValue, String description) {
+  OrcConf(String attribute,
+          String hiveConfName,
+          Object defaultValue,
+          String description) {
     this.attribute = attribute;
+    this.hiveConfName = hiveConfName;
     this.defaultValue = defaultValue;
     this.description = description;
   }
@@ -101,6 +118,10 @@ public enum OrcConf {
     return attribute;
   }
 
+  public String getHiveConfName() {
+    return hiveConfName;
+  }
+
   public Object getDefaultValue() {
     return defaultValue;
   }
@@ -109,26 +130,62 @@ public enum OrcConf {
     return description;
   }
 
+  private String lookupValue(Properties tbl, Configuration conf) {
+    String result = null;
+    if (tbl != null) {
+      result = conf.get(attribute);
+    }
+    if (result == null && conf != null) {
+      result = conf.get(attribute);
+      if (result == null) {
+        result = conf.get(hiveConfName);
+      }
+    }
+    return result;
+  }
+
+  public long getLong(Properties tbl, Configuration conf) {
+    String value = lookupValue(tbl, conf);
+    if (value != null) {
+      return Long.parseLong(value);
+    }
+    return ((Number) defaultValue).longValue();
+  }
+
   public long getLong(Configuration conf) {
-    return conf.getLong(attribute, ((Number) defaultValue).longValue());
+    return getLong(null, conf);
+  }
+
+  public String getString(Properties tbl, Configuration conf) {
+    String value = lookupValue(tbl, conf);
+    return value == null ? (String) defaultValue : value;
   }
 
   public String getString(Configuration conf) {
-    return conf.get(attribute, (String) defaultValue);
+    return getString(null, conf);
+  }
+
+  public boolean getBoolean(Properties tbl, Configuration conf) {
+    String value = lookupValue(tbl, conf);
+    if (value != null) {
+      return Boolean.parseBoolean(value);
+    }
+    return (Boolean) defaultValue;
   }
 
   public boolean getBoolean(Configuration conf) {
-    if (conf == null) {
-      return (Boolean) defaultValue;
+    return getBoolean(null, conf);
+  }
+
+  public double getDouble(Properties tbl, Configuration conf) {
+    String value = lookupValue(tbl, conf);
+    if (value != null) {
+      return Double.parseDouble(value);
     }
-    return conf.getBoolean(attribute, (Boolean) defaultValue);
+    return ((Number) defaultValue).doubleValue();
   }
 
   public double getDouble(Configuration conf) {
-    String str = conf.get(attribute);
-    if (str == null) {
-      return ((Number) defaultValue).doubleValue();
-    }
-    return Double.parseDouble(str);
+    return getDouble(null, conf);
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/cd2b4997/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFile.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFile.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFile.java
index 976a84b..2210769 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFile.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFile.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hive.ql.io.orc;
 
 import java.io.IOException;
+import java.util.Properties;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -47,7 +48,7 @@ public final class OrcFile {
    * prevent the new reader from reading ORC files generated by any released
    * version of Hive.
    */
-  public static enum Version {
+  public enum Version {
     V_0_11("0.11", 0, 11),
       V_0_12("0.12", 0, 12);
 
@@ -57,7 +58,7 @@ public final class OrcFile {
     private final int major;
     private final int minor;
 
-    private Version(String name, int major, int minor) {
+    Version(String name, int major, int minor) {
       this.name = name;
       this.major = major;
       this.minor = minor;
@@ -99,7 +100,7 @@ public final class OrcFile {
    * For bugs in the writer, but the old readers already read the new data
    * correctly, bump this version instead of the Version.
    */
-  public static enum WriterVersion {
+  public enum WriterVersion {
     ORIGINAL(0),
       HIVE_8732(1); // corrupted stripe/file maximum column statistics
 
@@ -109,55 +110,17 @@ public final class OrcFile {
       return id;
     }
 
-    private WriterVersion(int id) {
+    WriterVersion(int id) {
       this.id = id;
     }
   }
 
-  public static enum EncodingStrategy {
-    SPEED, COMPRESSION;
+  public enum EncodingStrategy {
+    SPEED, COMPRESSION
   }
 
-  public static enum CompressionStrategy {
-    SPEED, COMPRESSION;
-  }
-
-  // Note : these string definitions for table properties are deprecated,
-  // and retained only for backward compatibility, please do not add to
-  // them, add to OrcTableProperties below instead
-  @Deprecated public static final String COMPRESSION = "orc.compress";
-  @Deprecated public static final String COMPRESSION_BLOCK_SIZE = "orc.compress.size";
-  @Deprecated public static final String STRIPE_SIZE = "orc.stripe.size";
-  @Deprecated public static final String ROW_INDEX_STRIDE = "orc.row.index.stride";
-  @Deprecated public static final String ENABLE_INDEXES = "orc.create.index";
-  @Deprecated public static final String BLOCK_PADDING = "orc.block.padding";
-
-  /**
-   * Enum container for all orc table properties.
-   * If introducing a new orc-specific table property,
-   * add it here.
-   */
-  public static enum OrcTableProperties {
-    COMPRESSION("orc.compress"),
-    COMPRESSION_BLOCK_SIZE("orc.compress.size"),
-    STRIPE_SIZE("orc.stripe.size"),
-    BLOCK_SIZE("orc.block.size"),
-    ROW_INDEX_STRIDE("orc.row.index.stride"),
-    ENABLE_INDEXES("orc.create.index"),
-    BLOCK_PADDING("orc.block.padding"),
-    ENCODING_STRATEGY("orc.encoding.strategy"),
-    BLOOM_FILTER_COLUMNS("orc.bloom.filter.columns"),
-    BLOOM_FILTER_FPP("orc.bloom.filter.fpp");
-
-    private final String propName;
-
-    OrcTableProperties(String propName) {
-      this.propName = propName;
-    }
-
-    public String getPropName(){
-      return this.propName;
-    }
+  public enum CompressionStrategy {
+    SPEED, COMPRESSION
   }
 
   // unused
@@ -227,13 +190,13 @@ public final class OrcFile {
     return new ReaderImpl(path, options);
   }
 
-  public static interface WriterContext {
+  public interface WriterContext {
     Writer getWriter();
   }
 
-  public static interface WriterCallback {
-    public void preStripeWrite(WriterContext context) throws IOException;
-    public void preFooterWrite(WriterContext context) throws IOException;
+  public interface WriterCallback {
+    void preStripeWrite(WriterContext context) throws IOException;
+    void preFooterWrite(WriterContext context) throws IOException;
   }
 
   /**
@@ -258,40 +221,35 @@ public final class OrcFile {
     private String bloomFilterColumns;
     private double bloomFilterFpp;
 
-    WriterOptions(Configuration conf) {
+    WriterOptions(Properties tableProperties, Configuration conf) {
       configuration = conf;
       memoryManagerValue = getMemoryManager(conf);
-      stripeSizeValue = OrcConf.STRIPE_SIZE.getLong(conf);
-      blockSizeValue = OrcConf.BLOCK_SIZE.getLong(conf);
+      stripeSizeValue = OrcConf.STRIPE_SIZE.getLong(tableProperties, conf);
+      blockSizeValue = OrcConf.BLOCK_SIZE.getLong(tableProperties, conf);
       rowIndexStrideValue =
-          (int) OrcConf.ROW_INDEX_STRIDE.getLong(conf);
-      bufferSizeValue = (int) OrcConf.BUFFER_SIZE.getLong(conf);
-      blockPaddingValue = OrcConf.BLOCK_PADDING.getBoolean(conf);
+          (int) OrcConf.ROW_INDEX_STRIDE.getLong(tableProperties, conf);
+      bufferSizeValue = (int) OrcConf.BUFFER_SIZE.getLong(tableProperties,
+          conf);
+      blockPaddingValue =
+          OrcConf.BLOCK_PADDING.getBoolean(tableProperties, conf);
       compressValue =
-          CompressionKind.valueOf(OrcConf.COMPRESS.getString(conf));
-      String versionName = OrcConf.WRITE_FORMAT.getString(conf);
-      if (versionName == null) {
-        versionValue = Version.CURRENT;
-      } else {
-        versionValue = Version.byName(versionName);
-      }
-      String enString = OrcConf.ENCODING_STRATEGY.getString(conf);
-      if (enString == null) {
-        encodingStrategy = EncodingStrategy.SPEED;
-      } else {
-        encodingStrategy = EncodingStrategy.valueOf(enString);
-      }
-
-      String compString = OrcConf.COMPRESSION_STRATEGY.getString(conf);
-      if (compString == null) {
-        compressionStrategy = CompressionStrategy.SPEED;
-      } else {
-        compressionStrategy = CompressionStrategy.valueOf(compString);
-      }
+          CompressionKind.valueOf(OrcConf.COMPRESS.getString(tableProperties,
+              conf));
+      String versionName = OrcConf.WRITE_FORMAT.getString(tableProperties,
+          conf);
+      versionValue = Version.byName(versionName);
+      String enString = OrcConf.ENCODING_STRATEGY.getString(tableProperties,
+          conf);
+      encodingStrategy = EncodingStrategy.valueOf(enString);
+
+      String compString =
+          OrcConf.COMPRESSION_STRATEGY.getString(tableProperties, conf);
+      compressionStrategy = CompressionStrategy.valueOf(compString);
 
       paddingTolerance =
-          OrcConf.BLOCK_PADDING_TOLERANCE.getDouble(conf);
-      bloomFilterFpp = OrcConf.BLOOM_FILTER_FPP.getDouble(conf);
+          OrcConf.BLOCK_PADDING_TOLERANCE.getDouble(tableProperties, conf);
+      bloomFilterFpp = OrcConf.BLOOM_FILTER_FPP.getDouble(tableProperties,
+          conf);
     }
 
     /**
@@ -362,7 +320,7 @@ public final class OrcFile {
     /**
      * Sets the tolerance for block padding as a percentage of stripe size.
      */
-    public WriterOptions paddingTolerance(float value) {
+    public WriterOptions paddingTolerance(double value) {
       paddingTolerance = value;
       return this;
     }
@@ -378,7 +336,7 @@ public final class OrcFile {
     /**
      * Specify the false positive probability for bloom filter.
      * @param fpp - false positive probability
-     * @return
+     * @return this
      */
     public WriterOptions bloomFilterFpp(double fpp) {
       bloomFilterFpp = fpp;
@@ -413,7 +371,7 @@ public final class OrcFile {
     /**
      * Add a listener for when the stripe and file are about to be closed.
      * @param callback the object to be called when the stripe is closed
-     * @return
+     * @return this
      */
     public WriterOptions callback(WriterCallback callback) {
       this.callback = callback;
@@ -431,10 +389,24 @@ public final class OrcFile {
   }
 
   /**
-   * Create a default set of write options that can be modified.
+   * Create a set of writer options based on a configuration.
+   * @param conf the configuration to use for values
+   * @return A WriterOptions object that can be modified
    */
   public static WriterOptions writerOptions(Configuration conf) {
-    return new WriterOptions(conf);
+    return new WriterOptions(null, conf);
+  }
+
+  /**
+   * Create a set of write options based on a set of table properties and
+   * configuration.
+   * @param tableProperties the properties of the table
+   * @param conf the configuration of the query
+   * @return a WriterOptions object that can be modified
+   */
+  public static WriterOptions writerOptions(Properties tableProperties,
+                                            Configuration conf) {
+    return new WriterOptions(tableProperties, conf);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/cd2b4997/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcOutputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcOutputFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcOutputFormat.java
index 8625ff1..ea4ebb4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcOutputFormat.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcOutputFormat.java
@@ -114,73 +114,8 @@ public class OrcOutputFormat extends FileOutputFormat<NullWritable, OrcSerdeRow>
     }
   }
 
-  /**
-   * Helper method to get a parameter first from props if present, falling back to JobConf if not.
-   * Returns null if key is present in neither.
-   */
-  private String getSettingFromPropsFallingBackToConf(String key, Properties props, JobConf conf){
-    if ((props != null) && props.containsKey(key)){
-      return props.getProperty(key);
-    } else if(conf != null) {
-      // If conf is not null, and the key is not present, Configuration.get() will
-      // return null for us. So, we don't have to check if it contains it.
-      return conf.get(key);
-    } else {
-      return null;
-    }
-  }
-
   private OrcFile.WriterOptions getOptions(JobConf conf, Properties props) {
-    OrcFile.WriterOptions options = OrcFile.writerOptions(conf);
-    String propVal ;
-    if ((propVal = getSettingFromPropsFallingBackToConf(
-        OrcFile.OrcTableProperties.STRIPE_SIZE.getPropName(),props,conf)) != null){
-      options.stripeSize(Long.parseLong(propVal));
-    }
-
-    if ((propVal = getSettingFromPropsFallingBackToConf(
-        OrcFile.OrcTableProperties.COMPRESSION.getPropName(),props,conf)) != null){
-      options.compress(CompressionKind.valueOf(propVal));
-    }
-
-    if ((propVal = getSettingFromPropsFallingBackToConf(
-        OrcFile.OrcTableProperties.COMPRESSION_BLOCK_SIZE.getPropName(),props,conf)) != null){
-      options.bufferSize(Integer.parseInt(propVal));
-    }
-
-    if ((propVal = getSettingFromPropsFallingBackToConf(
-        OrcFile.OrcTableProperties.ROW_INDEX_STRIDE.getPropName(),props,conf)) != null){
-      options.rowIndexStride(Integer.parseInt(propVal));
-    }
-
-    if ((propVal = getSettingFromPropsFallingBackToConf(
-        OrcFile.OrcTableProperties.ENABLE_INDEXES.getPropName(),props,conf)) != null){
-      if ("false".equalsIgnoreCase(propVal)) {
-        options.rowIndexStride(0);
-      }
-    }
-
-    if ((propVal = getSettingFromPropsFallingBackToConf(
-        OrcFile.OrcTableProperties.BLOCK_PADDING.getPropName(),props,conf)) != null){
-      options.blockPadding(Boolean.parseBoolean(propVal));
-    }
-
-    if ((propVal = getSettingFromPropsFallingBackToConf(
-        OrcFile.OrcTableProperties.ENCODING_STRATEGY.getPropName(),props,conf)) != null){
-      options.encodingStrategy(EncodingStrategy.valueOf(propVal));
-    }
-
-    if ((propVal = getSettingFromPropsFallingBackToConf(
-        OrcFile.OrcTableProperties.BLOOM_FILTER_COLUMNS.getPropName(), props, conf)) != null) {
-      options.bloomFilterColumns(propVal);
-    }
-
-    if ((propVal = getSettingFromPropsFallingBackToConf(
-        OrcFile.OrcTableProperties.BLOOM_FILTER_FPP.getPropName(), props, conf)) != null) {
-      options.bloomFilterFpp(Double.parseDouble(propVal));
-    }
-
-    return options;
+    return OrcFile.writerOptions(props, conf);
   }
 
   @Override


[12/43] hive git commit: HIVE-11328: Avoid String representation of expression nodes in ConstantPropagateProcFactory unless necessary (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan/Hari Sankar Sivarama Subramaniyan)

Posted by xu...@apache.org.
HIVE-11328: Avoid String representation of expression nodes in ConstantPropagateProcFactory unless necessary (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan/Hari Sankar Sivarama Subramaniyan)


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

Branch: refs/heads/spark
Commit: 83be12fbb4d7cc7ef19779e1d14cac85023c8197
Parents: 72f97fc
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Wed Jul 22 10:08:42 2015 +0100
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Wed Jul 22 10:08:42 2015 +0100

----------------------------------------------------------------------
 .../optimizer/ConstantPropagateProcFactory.java | 100 ++++++++++++++-----
 1 file changed, 75 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/83be12fb/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java
index 286c042..410735c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java
@@ -155,7 +155,9 @@ public final class ConstantPropagateProcFactory {
       // ExprNodeConstantDesc
       return null;
     }
-    LOG.debug("Casting " + desc + " to type " + ti);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Casting " + desc + " to type " + ti);
+    }
     ExprNodeConstantDesc c = (ExprNodeConstantDesc) desc;
     if (null != c.getFoldedFromVal() && priti.getTypeName().equals(serdeConstants.STRING_TYPE_NAME)) {
       // avoid double casting to preserve original string representation of constant.
@@ -243,7 +245,9 @@ public final class ConstantPropagateProcFactory {
 
       // Don't evalulate nondeterministic function since the value can only calculate during runtime.
       if (!isDeterministicUdf(udf)) {
-        LOG.debug("Function " + udf.getClass() + " is undeterministic. Don't evalulating immediately.");
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Function " + udf.getClass() + " is undeterministic. Don't evalulating immediately.");
+        }
         ((ExprNodeGenericFuncDesc) desc).setChildren(newExprs);
         return desc;
       }
@@ -251,7 +255,9 @@ public final class ConstantPropagateProcFactory {
       // Check if the function can be short cut.
       ExprNodeDesc shortcut = shortcutFunction(udf, newExprs, op);
       if (shortcut != null) {
-        LOG.debug("Folding expression:" + desc + " -> " + shortcut);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Folding expression:" + desc + " -> " + shortcut);
+        }
         return shortcut;
       }
       ((ExprNodeGenericFuncDesc) desc).setChildren(newExprs);
@@ -293,20 +299,26 @@ public final class ConstantPropagateProcFactory {
 
       // Don't evalulate nondeterministic function since the value can only calculate during runtime.
       if (!isDeterministicUdf(udf)) {
-        LOG.debug("Function " + udf.getClass() + " is undeterministic. Don't evalulating immediately.");
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Function " + udf.getClass() + " is undeterministic. Don't evalulating immediately.");
+        }
         ((ExprNodeGenericFuncDesc) desc).setChildren(newExprs);
         return desc;
       } else {
         // If all child expressions of deterministic function are constants, evaluate such UDF immediately
         ExprNodeDesc constant = evaluateFunction(udf, newExprs, desc.getChildren());
         if (constant != null) {
-          LOG.debug("Folding expression:" + desc + " -> " + constant);
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Folding expression:" + desc + " -> " + constant);
+          }
           return constant;
         } else {
           // Check if the function can be short cut.
           ExprNodeDesc shortcut = shortcutFunction(udf, newExprs, op);
           if (shortcut != null) {
-            LOG.debug("Folding expression:" + desc + " -> " + shortcut);
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Folding expression:" + desc + " -> " + shortcut);
+            }
             return shortcut;
           }
           ((ExprNodeGenericFuncDesc) desc).setChildren(newExprs);
@@ -328,7 +340,9 @@ public final class ConstantPropagateProcFactory {
       Operator<? extends Serializable> parent = op.getParentOperators().get(tag);
       ExprNodeDesc col = evaluateColumn((ExprNodeColumnDesc) desc, cppCtx, parent);
       if (col != null) {
-        LOG.debug("Folding expression:" + desc + " -> " + col);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Folding expression:" + desc + " -> " + col);
+        }
         return col;
       }
     }
@@ -406,7 +420,9 @@ public final class ConstantPropagateProcFactory {
       }
       ColumnInfo ci = resolveColumn(rs, c);
       if (ci != null) {
-        LOG.debug("Filter " + udf + " is identified as a value assignment, propagate it.");
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Filter " + udf + " is identified as a value assignment, propagate it.");
+        }
         if (!v.getTypeInfo().equals(ci.getType())) {
           v = typeCast(v, ci.getType());
         }
@@ -417,7 +433,9 @@ public final class ConstantPropagateProcFactory {
     } else if (udf instanceof GenericUDFOPNull) {
       ExprNodeDesc operand = newExprs.get(0);
       if (operand instanceof ExprNodeColumnDesc) {
-        LOG.debug("Filter " + udf + " is identified as a value assignment, propagate it.");
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Filter " + udf + " is identified as a value assignment, propagate it.");
+        }
         ExprNodeColumnDesc c = (ExprNodeColumnDesc) operand;
         ColumnInfo ci = resolveColumn(rs, c);
         if (ci != null) {
@@ -641,11 +659,15 @@ public final class ConstantPropagateProcFactory {
     RowSchema rs = parent.getSchema();
     ColumnInfo ci = rs.getColumnInfo(desc.getColumn());
     if (ci == null) {
-      LOG.error("Reverse look up of column " + desc + " error!");
+      if (LOG.isErrorEnabled()) {
+        LOG.error("Reverse look up of column " + desc + " error!");
+      }
       ci = rs.getColumnInfo(desc.getTabAlias(), desc.getColumn());
     }
     if (ci == null) {
-      LOG.error("Can't resolve " + desc.getTabAlias() + "." + desc.getColumn());
+      if (LOG.isErrorEnabled()) {
+        LOG.error("Can't resolve " + desc.getTabAlias() + "." + desc.getColumn());
+      }
       return null;
     }
     ExprNodeDesc constant = null;
@@ -723,7 +745,9 @@ public final class ConstantPropagateProcFactory {
     try {
       ObjectInspector oi = udf.initialize(argois);
       Object o = udf.evaluate(arguments);
-      LOG.debug(udf.getClass().getName() + "(" + exprs + ")=" + o);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(udf.getClass().getName() + "(" + exprs + ")=" + o);
+      }
       if (o == null) {
         return new ExprNodeConstantDesc(TypeInfoUtils.getTypeInfoFromObjectInspector(oi), o);
       }
@@ -740,7 +764,9 @@ public final class ConstantPropagateProcFactory {
       } else if (PrimitiveObjectInspectorUtils.isPrimitiveJavaClass(clz)) {
 
       } else {
-        LOG.error("Unable to evaluate " + udf + ". Return value unrecoginizable.");
+        if (LOG.isErrorEnabled()) {
+          LOG.error("Unable to evaluate " + udf + ". Return value unrecoginizable.");
+        }
         return null;
       }
       String constStr = null;
@@ -771,7 +797,9 @@ public final class ConstantPropagateProcFactory {
       for (ColumnInfo col : schema.getSignature()) {
         ExprNodeDesc constant = constants.get(col);
         if (constant != null) {
-          LOG.debug("Replacing column " + col + " with constant " + constant + " in " + op);
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Replacing column " + col + " with constant " + constant + " in " + op);
+          }
           if (!col.getType().equals(constant.getTypeInfo())) {
             constant = typeCast(constant, col.getType());
           }
@@ -807,22 +835,30 @@ public final class ConstantPropagateProcFactory {
       cppCtx.getOpToConstantExprs().put(op, constants);
 
       ExprNodeDesc condn = op.getConf().getPredicate();
-      LOG.debug("Old filter FIL[" + op.getIdentifier() + "] conditions:" + condn.getExprString());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Old filter FIL[" + op.getIdentifier() + "] conditions:" + condn.getExprString());
+      }
       ExprNodeDesc newCondn = foldExpr(condn, constants, cppCtx, op, 0, true);
       if (newCondn instanceof ExprNodeConstantDesc) {
         ExprNodeConstantDesc c = (ExprNodeConstantDesc) newCondn;
         if (Boolean.TRUE.equals(c.getValue())) {
           cppCtx.addOpToDelete(op);
-          LOG.debug("Filter expression " + condn + " holds true. Will delete it.");
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Filter expression " + condn + " holds true. Will delete it.");
+          }
         } else if (Boolean.FALSE.equals(c.getValue())) {
-          LOG.warn("Filter expression " + condn + " holds false!");
+          if (LOG.isWarnEnabled()) {
+            LOG.warn("Filter expression " + condn + " holds false!");
+          }
         }
       }
       if (newCondn instanceof ExprNodeConstantDesc && ((ExprNodeConstantDesc)newCondn).getValue() == null) {
         // where null is same as where false
         newCondn = new ExprNodeConstantDesc(Boolean.FALSE);
       }
-      LOG.debug("New filter FIL[" + op.getIdentifier() + "] conditions:" + newCondn.getExprString());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("New filter FIL[" + op.getIdentifier() + "] conditions:" + newCondn.getExprString());
+      }
 
       // merge it with the downstream col list
       op.getConf().setPredicate(newCondn);
@@ -948,7 +984,9 @@ public final class ConstantPropagateProcFactory {
             columnExprMap.put(columnNames.get(i), newCol);
           }
         }
-        LOG.debug("New column list:(" + StringUtils.join(colList, " ") + ")");
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("New column list:(" + StringUtils.join(colList, " ") + ")");
+        }
       }
       return null;
     }
@@ -1026,7 +1064,9 @@ public final class ConstantPropagateProcFactory {
       Operator<?> op = (Operator<?>) nd;
       ConstantPropagateProcCtx cppCtx = (ConstantPropagateProcCtx) ctx;
       cppCtx.getOpToConstantExprs().put(op, new HashMap<ColumnInfo, ExprNodeDesc>());
-      LOG.debug("Stop propagate constants on op " + op.getOperatorId());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Stop propagate constants on op " + op.getOperatorId());
+      }
       return null;
     }
   }
@@ -1058,7 +1098,9 @@ public final class ConstantPropagateProcFactory {
           && op.getChildOperators().get(0) instanceof JoinOperator) {
         JoinOperator joinOp = (JoinOperator) op.getChildOperators().get(0);
         if (skipFolding(joinOp.getConf())) {
-          LOG.debug("Skip folding in outer join " + op);
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Skip folding in outer join " + op);
+          }
           cppCtx.getOpToConstantExprs().put(op, new HashMap<ColumnInfo, ExprNodeDesc>());
           return null;
         }
@@ -1066,7 +1108,9 @@ public final class ConstantPropagateProcFactory {
 
       if (rsDesc.getDistinctColumnIndices() != null
           && !rsDesc.getDistinctColumnIndices().isEmpty()) {
-        LOG.debug("Skip folding in distinct subqueries " + op);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Skip folding in distinct subqueries " + op);
+        }
         cppCtx.getOpToConstantExprs().put(op, new HashMap<ColumnInfo, ExprNodeDesc>());
         return null;
       }
@@ -1150,7 +1194,9 @@ public final class ConstantPropagateProcFactory {
         LOG.debug("Skip JOIN-RS structure.");
         return null;
       }
-      LOG.info("Old exprs " + conf.getExprs());
+      if (LOG.isInfoEnabled()) {
+        LOG.info("Old exprs " + conf.getExprs());
+      }
       Iterator<Entry<Byte, List<ExprNodeDesc>>> itr = conf.getExprs().entrySet().iterator();
       while (itr.hasNext()) {
         Entry<Byte, List<ExprNodeDesc>> e = itr.next();
@@ -1163,14 +1209,18 @@ public final class ConstantPropagateProcFactory {
         for (ExprNodeDesc expr : exprs) {
           ExprNodeDesc newExpr = foldExpr(expr, constants, cppCtx, op, tag, false);
           if (newExpr instanceof ExprNodeConstantDesc) {
-            LOG.info("expr " + newExpr + " fold from " + expr + " is removed.");
+            if (LOG.isInfoEnabled()) {
+              LOG.info("expr " + newExpr + " fold from " + expr + " is removed.");
+            }
             continue;
           }
           newExprs.add(newExpr);
         }
         e.setValue(newExprs);
       }
-      LOG.info("New exprs " + conf.getExprs());
+      if (LOG.isInfoEnabled()) {
+        LOG.info("New exprs " + conf.getExprs());
+      }
 
       for (List<ExprNodeDesc> v : conf.getFilters().values()) {
         for (int i = 0; i < v.size(); i++) {


[37/43] hive git commit: HIVE-11253. Move SearchArgument and VectorizedRowBatch classes to storage-api. (omalley reviewed by prasanthj)

Posted by xu...@apache.org.
HIVE-11253. Move SearchArgument and VectorizedRowBatch classes to storage-api.
(omalley reviewed by prasanthj)


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

Branch: refs/heads/spark
Commit: 9ae70cb4d11dae6cea45c29b0e87dc5da1e5555c
Parents: 70b56e3
Author: Owen O'Malley <om...@apache.org>
Authored: Wed Jul 29 09:36:08 2015 -0700
Committer: Owen O'Malley <om...@apache.org>
Committed: Wed Jul 29 09:36:08 2015 -0700

----------------------------------------------------------------------
 common/pom.xml                                  |   5 +
 .../hadoop/hive/common/type/HiveDecimal.java    | 312 ---------
 pom.xml                                         |   1 +
 .../hive/ql/exec/vector/BytesColumnVector.java  | 322 ---------
 .../hive/ql/exec/vector/ColumnVector.java       | 174 -----
 .../ql/exec/vector/DecimalColumnVector.java     | 106 ---
 .../hive/ql/exec/vector/DoubleColumnVector.java | 143 ----
 .../hive/ql/exec/vector/LongColumnVector.java   | 189 -----
 .../hive/ql/exec/vector/VectorizedRowBatch.java | 186 -----
 .../hive/ql/io/sarg/SearchArgumentFactory.java  |  39 --
 .../hive/ql/io/sarg/SearchArgumentImpl.java     | 697 -------------------
 .../hive/ql/io/orc/TestInputOutputFormat.java   |  17 +-
 .../hive/ql/io/sarg/TestSearchArgumentImpl.java |   3 +-
 .../hadoop/hive/ql/io/sarg/ExpressionTree.java  | 157 -----
 .../hadoop/hive/ql/io/sarg/PredicateLeaf.java   | 104 ---
 .../hadoop/hive/ql/io/sarg/SearchArgument.java  | 298 --------
 .../hive/serde2/io/HiveDecimalWritable.java     | 174 -----
 storage-api/pom.xml                             |  85 +++
 .../hadoop/hive/common/type/HiveDecimal.java    | 312 +++++++++
 .../hive/ql/exec/vector/BytesColumnVector.java  | 322 +++++++++
 .../hive/ql/exec/vector/ColumnVector.java       | 173 +++++
 .../ql/exec/vector/DecimalColumnVector.java     | 106 +++
 .../hive/ql/exec/vector/DoubleColumnVector.java | 143 ++++
 .../hive/ql/exec/vector/LongColumnVector.java   | 189 +++++
 .../hive/ql/exec/vector/VectorizedRowBatch.java | 186 +++++
 .../hadoop/hive/ql/io/sarg/ExpressionTree.java  | 156 +++++
 .../hadoop/hive/ql/io/sarg/PredicateLeaf.java   | 104 +++
 .../hadoop/hive/ql/io/sarg/SearchArgument.java  | 287 ++++++++
 .../hive/ql/io/sarg/SearchArgumentFactory.java  |  28 +
 .../hive/ql/io/sarg/SearchArgumentImpl.java     | 687 ++++++++++++++++++
 .../hive/serde2/io/HiveDecimalWritable.java     | 174 +++++
 31 files changed, 2974 insertions(+), 2905 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/9ae70cb4/common/pom.xml
----------------------------------------------------------------------
diff --git a/common/pom.xml b/common/pom.xml
index aedf7ba..a7997e2 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -39,6 +39,11 @@
       <artifactId>hive-shims</artifactId>
       <version>${project.version}</version>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hive</groupId>
+      <artifactId>hive-storage-api</artifactId>
+      <version>${project.version}</version>
+    </dependency>
     <!-- inter-project -->
     <dependency>
       <groupId>commons-cli</groupId>

http://git-wip-us.apache.org/repos/asf/hive/blob/9ae70cb4/common/src/java/org/apache/hadoop/hive/common/type/HiveDecimal.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/type/HiveDecimal.java b/common/src/java/org/apache/hadoop/hive/common/type/HiveDecimal.java
deleted file mode 100644
index 7d7fb28..0000000
--- a/common/src/java/org/apache/hadoop/hive/common/type/HiveDecimal.java
+++ /dev/null
@@ -1,312 +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.
- */
-package org.apache.hadoop.hive.common.type;
-
-import java.math.BigDecimal;
-import java.math.BigInteger;
-import java.math.RoundingMode;
-
-/**
- *
- * HiveDecimal. Simple wrapper for BigDecimal. Adds fixed max precision and non scientific string
- * representation
- *
- */
-public class HiveDecimal implements Comparable<HiveDecimal> {
-  public static final int MAX_PRECISION = 38;
-  public static final int MAX_SCALE = 38;
-
-  /**
-   * Default precision/scale when user doesn't specify in the column metadata, such as
-   * decimal and decimal(8).
-   */
-  public static final int USER_DEFAULT_PRECISION = 10;
-  public static final int USER_DEFAULT_SCALE = 0;
-
-  /**
-   *  Default precision/scale when system is not able to determine them, such as in case
-   *  of a non-generic udf.
-   */
-  public static final int SYSTEM_DEFAULT_PRECISION = 38;
-  public static final int SYSTEM_DEFAULT_SCALE = 18;
-
-  public static final HiveDecimal ZERO = new HiveDecimal(BigDecimal.ZERO);
-  public static final HiveDecimal ONE = new HiveDecimal(BigDecimal.ONE);
-
-  public static final int ROUND_FLOOR = BigDecimal.ROUND_FLOOR;
-  public static final int ROUND_CEILING = BigDecimal.ROUND_CEILING;
-  public static final int ROUND_HALF_UP = BigDecimal.ROUND_HALF_UP;
-
-  private BigDecimal bd = BigDecimal.ZERO;
-
-  private HiveDecimal(BigDecimal bd) {
-    this.bd = bd;
-  }
-
-  public static HiveDecimal create(BigDecimal b) {
-    return create(b, true);
-  }
-
-  public static HiveDecimal create(BigDecimal b, boolean allowRounding) {
-    BigDecimal bd = normalize(b, allowRounding);
-    return bd == null ? null : new HiveDecimal(bd);
-  }
-
-  public static HiveDecimal create(BigInteger unscaled, int scale) {
-    BigDecimal bd = normalize(new BigDecimal(unscaled, scale), true);
-    return bd == null ? null : new HiveDecimal(bd);
-  }
-
-  public static HiveDecimal create(String dec) {
-    BigDecimal bd;
-    try {
-      bd = new BigDecimal(dec.trim());
-    } catch (NumberFormatException ex) {
-      return null;
-    }
-
-    bd = normalize(bd, true);
-    return bd == null ? null : new HiveDecimal(bd);
-  }
-
-  public static HiveDecimal create(BigInteger bi) {
-    BigDecimal bd = normalize(new BigDecimal(bi), true);
-    return bd == null ? null : new HiveDecimal(bd);
-  }
-
-  public static HiveDecimal create(int i) {
-    return new HiveDecimal(new BigDecimal(i));
-  }
-
-  public static HiveDecimal create(long l) {
-    return new HiveDecimal(new BigDecimal(l));
-  }
-
-  @Override
-  public String toString() {
-     return bd.toPlainString();
-  }
-
-  public HiveDecimal setScale(int i) {
-    return new HiveDecimal(bd.setScale(i, RoundingMode.HALF_UP));
-  }
-
-  @Override
-  public int compareTo(HiveDecimal dec) {
-    return bd.compareTo(dec.bd);
-  }
-
-  @Override
-  public int hashCode() {
-    return bd.hashCode();
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj == null || obj.getClass() != getClass()) {
-      return false;
-    }
-    return bd.equals(((HiveDecimal) obj).bd);
-  }
-
-  public int scale() {
-    return bd.scale();
-  }
-
-  /**
-   * Returns the number of digits (integer and fractional) in the number, which is equivalent
-   * to SQL decimal precision. Note that this is different from BigDecimal.precision(),
-   * which returns the precision of the unscaled value (BigDecimal.valueOf(0.01).precision() = 1,
-   * whereas HiveDecimal.create("0.01").precision() = 2).
-   * If you want the BigDecimal precision, use HiveDecimal.bigDecimalValue().precision()
-   * @return
-   */
-  public int precision() {
-    int bdPrecision = bd.precision();
-    int bdScale = bd.scale();
-
-    if (bdPrecision < bdScale) {
-      // This can happen for numbers less than 0.1
-      // For 0.001234: bdPrecision=4, bdScale=6
-      // In this case, we'll set the type to have the same precision as the scale.
-      return bdScale;
-    }
-    return bdPrecision;
-  }
-
-  public int intValue() {
-    return bd.intValue();
-  }
-
-  public double doubleValue() {
-    return bd.doubleValue();
-  }
-
-  public long longValue() {
-    return bd.longValue();
-  }
-
-  public short shortValue() {
-    return bd.shortValue();
-  }
-
-  public float floatValue() {
-    return bd.floatValue();
-  }
-
-  public BigDecimal bigDecimalValue() {
-    return bd;
-  }
-
-  public byte byteValue() {
-    return bd.byteValue();
-  }
-
-  public HiveDecimal setScale(int adjustedScale, int rm) {
-    return create(bd.setScale(adjustedScale, rm));
-  }
-
-  public HiveDecimal subtract(HiveDecimal dec) {
-    return create(bd.subtract(dec.bd));
-  }
-
-  public HiveDecimal multiply(HiveDecimal dec) {
-    return create(bd.multiply(dec.bd), false);
-  }
-
-  public BigInteger unscaledValue() {
-    return bd.unscaledValue();
-  }
-
-  public HiveDecimal scaleByPowerOfTen(int n) {
-    return create(bd.scaleByPowerOfTen(n));
-  }
-
-  public HiveDecimal abs() {
-    return create(bd.abs());
-  }
-
-  public HiveDecimal negate() {
-    return create(bd.negate());
-  }
-
-  public HiveDecimal add(HiveDecimal dec) {
-    return create(bd.add(dec.bd));
-  }
-
-  public HiveDecimal pow(int n) {
-    BigDecimal result = normalize(bd.pow(n), false);
-    return result == null ? null : new HiveDecimal(result);
-  }
-
-  public HiveDecimal remainder(HiveDecimal dec) {
-    return create(bd.remainder(dec.bd));
-  }
-
-  public HiveDecimal divide(HiveDecimal dec) {
-    return create(bd.divide(dec.bd, MAX_SCALE, RoundingMode.HALF_UP), true);
-  }
-
-  /**
-   * Get the sign of the underlying decimal.
-   * @return 0 if the decimal is equal to 0, -1 if less than zero, and 1 if greater than 0
-   */
-  public int signum() {
-    return bd.signum();
-  }
-
-  private static BigDecimal trim(BigDecimal d) {
-    if (d.compareTo(BigDecimal.ZERO) == 0) {
-      // Special case for 0, because java doesn't strip zeros correctly on that number.
-      d = BigDecimal.ZERO;
-    } else {
-      d = d.stripTrailingZeros();
-      if (d.scale() < 0) {
-        // no negative scale decimals
-        d = d.setScale(0);
-      }
-    }
-    return d;
-  }
-
-  private static BigDecimal normalize(BigDecimal bd, boolean allowRounding) {
-    if (bd == null) {
-      return null;
-    }
-
-    bd = trim(bd);
-
-    int intDigits = bd.precision() - bd.scale();
-
-    if (intDigits > MAX_PRECISION) {
-      return null;
-    }
-
-    int maxScale = Math.min(MAX_SCALE, Math.min(MAX_PRECISION - intDigits, bd.scale()));
-    if (bd.scale() > maxScale ) {
-      if (allowRounding) {
-        bd = bd.setScale(maxScale, RoundingMode.HALF_UP);
-        // Trimming is again necessary, because rounding may introduce new trailing 0's.
-        bd = trim(bd);
-      } else {
-        bd = null;
-      }
-    }
-
-    return bd;
-  }
-
-  public static BigDecimal enforcePrecisionScale(BigDecimal bd, int maxPrecision, int maxScale) {
-    if (bd == null) {
-      return null;
-    }
-
-    bd = trim(bd);
-
-    if (bd.scale() > maxScale) {
-      bd = bd.setScale(maxScale, RoundingMode.HALF_UP);
-    }
-
-    int maxIntDigits = maxPrecision - maxScale;
-    int intDigits = bd.precision() - bd.scale();
-    if (intDigits > maxIntDigits) {
-      return null;
-    }
-
-    return bd;
-  }
-
-  public static HiveDecimal enforcePrecisionScale(HiveDecimal dec, int maxPrecision, int maxScale) {
-    if (dec == null) {
-      return null;
-    }
-
-    // Minor optimization, avoiding creating new objects.
-    if (dec.precision() - dec.scale() <= maxPrecision - maxScale &&
-        dec.scale() <= maxScale) {
-      return dec;
-    }
-
-    BigDecimal bd = enforcePrecisionScale(dec.bd, maxPrecision, maxScale);
-    if (bd == null) {
-      return null;
-    }
-
-    return HiveDecimal.create(bd);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/9ae70cb4/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 1abf738..196a0a4 100644
--- a/pom.xml
+++ b/pom.xml
@@ -49,6 +49,7 @@
     <module>service</module>
     <module>shims</module>
     <module>spark-client</module>
+    <module>storage-api</module>
     <module>testutils</module>
     <module>packaging</module>
   </modules>

http://git-wip-us.apache.org/repos/asf/hive/blob/9ae70cb4/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/BytesColumnVector.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/BytesColumnVector.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/BytesColumnVector.java
deleted file mode 100644
index 02c52fa..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/BytesColumnVector.java
+++ /dev/null
@@ -1,322 +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.
- */
-
-package org.apache.hadoop.hive.ql.exec.vector;
-
-/**
- * This class supports string and binary data by value reference -- i.e. each field is
- * explicitly present, as opposed to provided by a dictionary reference.
- * In some cases, all the values will be in the same byte array to begin with,
- * but this need not be the case. If each value is in a separate byte
- * array to start with, or not all of the values are in the same original
- * byte array, you can still assign data by reference into this column vector.
- * This gives flexibility to use this in multiple situations.
- * <p>
- * When setting data by reference, the caller
- * is responsible for allocating the byte arrays used to hold the data.
- * You can also set data by value, as long as you call the initBuffer() method first.
- * You can mix "by value" and "by reference" in the same column vector,
- * though that use is probably not typical.
- */
-public class BytesColumnVector extends ColumnVector {
-  public byte[][] vector;
-  public int[] start;          // start offset of each field
-
-  /*
-   * The length of each field. If the value repeats for every entry, then it is stored
-   * in vector[0] and isRepeating from the superclass is set to true.
-   */
-  public int[] length;
-  private byte[] buffer;   // optional buffer to use when actually copying in data
-  private int nextFree;    // next free position in buffer
-
-  // Estimate that there will be 16 bytes per entry
-  static final int DEFAULT_BUFFER_SIZE = 16 * VectorizedRowBatch.DEFAULT_SIZE;
-
-  // Proportion of extra space to provide when allocating more buffer space.
-  static final float EXTRA_SPACE_FACTOR = (float) 1.2;
-
-  /**
-   * Use this constructor for normal operation.
-   * All column vectors should be the default size normally.
-   */
-  public BytesColumnVector() {
-    this(VectorizedRowBatch.DEFAULT_SIZE);
-  }
-
-  /**
-   * Don't call this constructor except for testing purposes.
-   *
-   * @param size  number of elements in the column vector
-   */
-  public BytesColumnVector(int size) {
-    super(size);
-    vector = new byte[size][];
-    start = new int[size];
-    length = new int[size];
-  }
-
-  /**
-   * Additional reset work for BytesColumnVector (releasing scratch bytes for by value strings).
-   */
-  @Override
-  public void reset() {
-    super.reset();
-    initBuffer(0);
-  }
-
-  /** Set a field by reference.
-   *
-   * @param elementNum index within column vector to set
-   * @param sourceBuf container of source data
-   * @param start start byte position within source
-   * @param length  length of source byte sequence
-   */
-  public void setRef(int elementNum, byte[] sourceBuf, int start, int length) {
-    vector[elementNum] = sourceBuf;
-    this.start[elementNum] = start;
-    this.length[elementNum] = length;
-  }
-
-  /**
-   * You must call initBuffer first before using setVal().
-   * Provide the estimated number of bytes needed to hold
-   * a full column vector worth of byte string data.
-   *
-   * @param estimatedValueSize  Estimated size of buffer space needed
-   */
-  public void initBuffer(int estimatedValueSize) {
-    nextFree = 0;
-
-    // if buffer is already allocated, keep using it, don't re-allocate
-    if (buffer != null) {
-      return;
-    }
-
-    // allocate a little extra space to limit need to re-allocate
-    int bufferSize = this.vector.length * (int)(estimatedValueSize * EXTRA_SPACE_FACTOR);
-    if (bufferSize < DEFAULT_BUFFER_SIZE) {
-      bufferSize = DEFAULT_BUFFER_SIZE;
-    }
-    buffer = new byte[bufferSize];
-  }
-
-  /**
-   * Initialize buffer to default size.
-   */
-  public void initBuffer() {
-    initBuffer(0);
-  }
-
-  /**
-   * @return amount of buffer space currently allocated
-   */
-  public int bufferSize() {
-    if (buffer == null) {
-      return 0;
-    }
-    return buffer.length;
-  }
-
-  /**
-   * Set a field by actually copying in to a local buffer.
-   * If you must actually copy data in to the array, use this method.
-   * DO NOT USE this method unless it's not practical to set data by reference with setRef().
-   * Setting data by reference tends to run a lot faster than copying data in.
-   *
-   * @param elementNum index within column vector to set
-   * @param sourceBuf container of source data
-   * @param start start byte position within source
-   * @param length  length of source byte sequence
-   */
-  public void setVal(int elementNum, byte[] sourceBuf, int start, int length) {
-    if ((nextFree + length) > buffer.length) {
-      increaseBufferSpace(length);
-    }
-    System.arraycopy(sourceBuf, start, buffer, nextFree, length);
-    vector[elementNum] = buffer;
-    this.start[elementNum] = nextFree;
-    this.length[elementNum] = length;
-    nextFree += length;
-  }
-
-  /**
-   * Set a field to the concatenation of two string values. Result data is copied
-   * into the internal buffer.
-   *
-   * @param elementNum index within column vector to set
-   * @param leftSourceBuf container of left argument
-   * @param leftStart start of left argument
-   * @param leftLen length of left argument
-   * @param rightSourceBuf container of right argument
-   * @param rightStart start of right argument
-   * @param rightLen length of right arugment
-   */
-  public void setConcat(int elementNum, byte[] leftSourceBuf, int leftStart, int leftLen,
-      byte[] rightSourceBuf, int rightStart, int rightLen) {
-    int newLen = leftLen + rightLen;
-    if ((nextFree + newLen) > buffer.length) {
-      increaseBufferSpace(newLen);
-    }
-    vector[elementNum] = buffer;
-    this.start[elementNum] = nextFree;
-    this.length[elementNum] = newLen;
-
-    System.arraycopy(leftSourceBuf, leftStart, buffer, nextFree, leftLen);
-    nextFree += leftLen;
-    System.arraycopy(rightSourceBuf, rightStart, buffer, nextFree, rightLen);
-    nextFree += rightLen;
-  }
-
-  /**
-   * Increase buffer space enough to accommodate next element.
-   * This uses an exponential increase mechanism to rapidly
-   * increase buffer size to enough to hold all data.
-   * As batches get re-loaded, buffer space allocated will quickly
-   * stabilize.
-   *
-   * @param nextElemLength size of next element to be added
-   */
-  public void increaseBufferSpace(int nextElemLength) {
-
-    // Keep doubling buffer size until there will be enough space for next element.
-    int newLength = 2 * buffer.length;
-    while((nextFree + nextElemLength) > newLength) {
-      newLength *= 2;
-    }
-
-    // Allocate new buffer, copy data to it, and set buffer to new buffer.
-    byte[] newBuffer = new byte[newLength];
-    System.arraycopy(buffer, 0, newBuffer, 0, nextFree);
-    buffer = newBuffer;
-  }
-
-  /** Copy the current object contents into the output. Only copy selected entries,
-    * as indicated by selectedInUse and the sel array.
-    */
-  public void copySelected(
-      boolean selectedInUse, int[] sel, int size, BytesColumnVector output) {
-
-    // Output has nulls if and only if input has nulls.
-    output.noNulls = noNulls;
-    output.isRepeating = false;
-
-    // Handle repeating case
-    if (isRepeating) {
-      output.setVal(0, vector[0], start[0], length[0]);
-      output.isNull[0] = isNull[0];
-      output.isRepeating = true;
-      return;
-    }
-
-    // Handle normal case
-
-    // Copy data values over
-    if (selectedInUse) {
-      for (int j = 0; j < size; j++) {
-        int i = sel[j];
-        output.setVal(i, vector[i], start[i], length[i]);
-      }
-    }
-    else {
-      for (int i = 0; i < size; i++) {
-        output.setVal(i, vector[i], start[i], length[i]);
-      }
-    }
-
-    // Copy nulls over if needed
-    if (!noNulls) {
-      if (selectedInUse) {
-        for (int j = 0; j < size; j++) {
-          int i = sel[j];
-          output.isNull[i] = isNull[i];
-        }
-      }
-      else {
-        System.arraycopy(isNull, 0, output.isNull, 0, size);
-      }
-    }
-  }
-
-  /** Simplify vector by brute-force flattening noNulls and isRepeating
-    * This can be used to reduce combinatorial explosion of code paths in VectorExpressions
-    * with many arguments, at the expense of loss of some performance.
-    */
-  public void flatten(boolean selectedInUse, int[] sel, int size) {
-    flattenPush();
-    if (isRepeating) {
-      isRepeating = false;
-
-      // setRef is used below and this is safe, because the reference
-      // is to data owned by this column vector. If this column vector
-      // gets re-used, the whole thing is re-used together so there
-      // is no danger of a dangling reference.
-
-      // Only copy data values if entry is not null. The string value
-      // at position 0 is undefined if the position 0 value is null.
-      if (noNulls || !isNull[0]) {
-
-        // loops start at position 1 because position 0 is already set
-        if (selectedInUse) {
-          for (int j = 1; j < size; j++) {
-            int i = sel[j];
-            this.setRef(i, vector[0], start[0], length[0]);
-          }
-        } else {
-          for (int i = 1; i < size; i++) {
-            this.setRef(i, vector[0], start[0], length[0]);
-          }
-        }
-      }
-      flattenRepeatingNulls(selectedInUse, sel, size);
-    }
-    flattenNoNulls(selectedInUse, sel, size);
-  }
-
-  // Fill the all the vector entries with provided value
-  public void fill(byte[] value) {
-    noNulls = true;
-    isRepeating = true;
-    setRef(0, value, 0, value.length);
-  }
-
-  @Override
-  public void setElement(int outElementNum, int inputElementNum, ColumnVector inputVector) {
-    BytesColumnVector in = (BytesColumnVector) inputVector;
-    setVal(outElementNum, in.vector[inputElementNum], in.start[inputElementNum], in.length[inputElementNum]);
-  }
-
-  @Override
-  public void init() {
-    initBuffer(0);
-  }
-
-  @Override
-  public void stringifyValue(StringBuilder buffer, int row) {
-    if (isRepeating) {
-      row = 0;
-    }
-    if (noNulls || !isNull[row]) {
-      buffer.append('"');
-      buffer.append(new String(this.buffer, start[row], length[row]));
-      buffer.append('"');
-    } else {
-      buffer.append("null");
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/9ae70cb4/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ColumnVector.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ColumnVector.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ColumnVector.java
deleted file mode 100644
index 4b5cf39..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ColumnVector.java
+++ /dev/null
@@ -1,174 +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.
- */
-
-package org.apache.hadoop.hive.ql.exec.vector;
-
-import java.util.Arrays;
-
-/**
- * ColumnVector contains the shared structure for the sub-types,
- * including NULL information, and whether this vector
- * repeats, i.e. has all values the same, so only the first
- * one is set. This is used to accelerate query performance
- * by handling a whole vector in O(1) time when applicable.
- *
- * The fields are public by design since this is a performance-critical
- * structure that is used in the inner loop of query execution.
- */
-public abstract class ColumnVector {
-
-  /*
-   * The current kinds of column vectors.
-   */
-  public static enum Type {
-    LONG,
-    DOUBLE,
-    BYTES,
-    DECIMAL
-  }
-
-  /*
-   * If hasNulls is true, then this array contains true if the value
-   * is null, otherwise false. The array is always allocated, so a batch can be re-used
-   * later and nulls added.
-   */
-  public boolean[] isNull;
-
-  // If the whole column vector has no nulls, this is true, otherwise false.
-  public boolean noNulls;
-
-  /*
-   * True if same value repeats for whole column vector.
-   * If so, vector[0] holds the repeating value.
-   */
-  public boolean isRepeating;
-
-  // Variables to hold state from before flattening so it can be easily restored.
-  private boolean preFlattenIsRepeating;
-  private boolean preFlattenNoNulls;
-
-  /**
-   * Constructor for super-class ColumnVector. This is not called directly,
-   * but used to initialize inherited fields.
-   *
-   * @param len Vector length
-   */
-  public ColumnVector(int len) {
-    isNull = new boolean[len];
-    noNulls = true;
-    isRepeating = false;
-  }
-
-  /**
-     * Resets the column to default state
-     *  - fills the isNull array with false
-     *  - sets noNulls to true
-     *  - sets isRepeating to false
-     */
-    public void reset() {
-      if (false == noNulls) {
-        Arrays.fill(isNull, false);
-      }
-      noNulls = true;
-      isRepeating = false;
-    }
-
-    abstract public void flatten(boolean selectedInUse, int[] sel, int size);
-
-    // Simplify vector by brute-force flattening noNulls if isRepeating
-    // This can be used to reduce combinatorial explosion of code paths in VectorExpressions
-    // with many arguments.
-    public void flattenRepeatingNulls(boolean selectedInUse, int[] sel, int size) {
-
-      boolean nullFillValue;
-
-      if (noNulls) {
-        nullFillValue = false;
-      } else {
-        nullFillValue = isNull[0];
-      }
-
-      if (selectedInUse) {
-        for (int j = 0; j < size; j++) {
-          int i = sel[j];
-          isNull[i] = nullFillValue;
-        }
-      } else {
-        Arrays.fill(isNull, 0, size, nullFillValue);
-      }
-
-      // all nulls are now explicit
-      noNulls = false;
-    }
-
-    public void flattenNoNulls(boolean selectedInUse, int[] sel, int size) {
-      if (noNulls) {
-        noNulls = false;
-        if (selectedInUse) {
-          for (int j = 0; j < size; j++) {
-            int i = sel[j];
-            isNull[i] = false;
-          }
-        } else {
-          Arrays.fill(isNull, 0, size, false);
-        }
-      }
-    }
-
-    /**
-     * Restore the state of isRepeating and noNulls to what it was
-     * before flattening. This must only be called just after flattening
-     * and then evaluating a VectorExpression on the column vector.
-     * It is an optimization that allows other operations on the same
-     * column to continue to benefit from the isRepeating and noNulls
-     * indicators.
-     */
-    public void unFlatten() {
-      isRepeating = preFlattenIsRepeating;
-      noNulls = preFlattenNoNulls;
-    }
-
-    // Record repeating and no nulls state to be restored later.
-    protected void flattenPush() {
-      preFlattenIsRepeating = isRepeating;
-      preFlattenNoNulls = noNulls;
-    }
-
-    /**
-     * Set the element in this column vector from the given input vector.
-     */
-    public abstract void setElement(int outElementNum, int inputElementNum, ColumnVector inputVector);
-
-    /**
-     * Initialize the column vector. This method can be overridden by specific column vector types.
-     * Use this method only if the individual type of the column vector is not known, otherwise its
-     * preferable to call specific initialization methods.
-     */
-    public void init() {
-      // Do nothing by default
-    }
-
-    /**
-     * Print the value for this column into the given string builder.
-     * @param buffer the buffer to print into
-     * @param row the id of the row to print
-     */
-    public abstract void stringifyValue(StringBuilder buffer,
-                                        int row);
-  }
-

http://git-wip-us.apache.org/repos/asf/hive/blob/9ae70cb4/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/DecimalColumnVector.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/DecimalColumnVector.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/DecimalColumnVector.java
deleted file mode 100644
index 74a9d5f..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/DecimalColumnVector.java
+++ /dev/null
@@ -1,106 +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.
- */
-
-package org.apache.hadoop.hive.ql.exec.vector;
-
-import java.math.BigInteger;
-
-import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
-import org.apache.hadoop.hive.common.type.HiveDecimal;
-
-public class DecimalColumnVector extends ColumnVector {
-
-  /**
-   * A vector of HiveDecimalWritable objects.
-   *
-   * For high performance and easy access to this low-level structure,
-   * the fields are public by design (as they are in other ColumnVector
-   * types).
-   */
-  public HiveDecimalWritable[] vector;
-  public short scale;
-  public short precision;
-
-  public DecimalColumnVector(int precision, int scale) {
-    this(VectorizedRowBatch.DEFAULT_SIZE, precision, scale);
-  }
-
-  public DecimalColumnVector(int size, int precision, int scale) {
-    super(size);
-    this.precision = (short) precision;
-    this.scale = (short) scale;
-    vector = new HiveDecimalWritable[size];
-    for (int i = 0; i < size; i++) {
-      vector[i] = new HiveDecimalWritable(HiveDecimal.ZERO);
-    }
-  }
-
-  @Override
-  public void flatten(boolean selectedInUse, int[] sel, int size) {
-    // TODO Auto-generated method stub
-  }
-
-  @Override
-  public void setElement(int outElementNum, int inputElementNum, ColumnVector inputVector) {
-    HiveDecimal hiveDec = ((DecimalColumnVector) inputVector).vector[inputElementNum].getHiveDecimal(precision, scale);
-    if (hiveDec == null) {
-      noNulls = false;
-      isNull[outElementNum] = true;
-    } else {
-      vector[outElementNum].set(hiveDec);
-    }
-  }
-
-  @Override
-  public void stringifyValue(StringBuilder buffer, int row) {
-    if (isRepeating) {
-      row = 0;
-    }
-    if (noNulls || !isNull[row]) {
-      buffer.append(vector[row].toString());
-    } else {
-      buffer.append("null");
-    }
-  }
-
-  public void set(int elementNum, HiveDecimalWritable writeable) {
-    HiveDecimal hiveDec = writeable.getHiveDecimal(precision, scale);
-    if (hiveDec == null) {
-      noNulls = false;
-      isNull[elementNum] = true;
-    } else {
-      vector[elementNum].set(hiveDec);
-    }
-  }
-
-  public void set(int elementNum, HiveDecimal hiveDec) {
-    HiveDecimal checkedDec = HiveDecimal.enforcePrecisionScale(hiveDec, precision, scale);
-    if (checkedDec == null) {
-      noNulls = false;
-      isNull[elementNum] = true;
-    } else {
-      vector[elementNum].set(checkedDec);
-    }
-  }
-
-  public void setNullDataValue(int elementNum) {
-    // E.g. For scale 2 the minimum is "0.01"
-    HiveDecimal minimumNonZeroValue = HiveDecimal.create(BigInteger.ONE, scale);
-    vector[elementNum].set(minimumNonZeroValue);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/9ae70cb4/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/DoubleColumnVector.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/DoubleColumnVector.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/DoubleColumnVector.java
deleted file mode 100644
index 4a7811d..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/DoubleColumnVector.java
+++ /dev/null
@@ -1,143 +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.
- */
-package org.apache.hadoop.hive.ql.exec.vector;
-
-import java.util.Arrays;
-
-/**
- * This class represents a nullable double precision floating point column vector.
- * This class will be used for operations on all floating point types (float, double)
- * and as such will use a 64-bit double value to hold the biggest possible value.
- * During copy-in/copy-out, smaller types (i.e. float) will be converted as needed. This will
- * reduce the amount of code that needs to be generated and also will run fast since the
- * machine operates with 64-bit words.
- *
- * The vector[] field is public by design for high-performance access in the inner
- * loop of query execution.
- */
-public class DoubleColumnVector extends ColumnVector {
-  public double[] vector;
-  public static final double NULL_VALUE = Double.NaN;
-
-  /**
-   * Use this constructor by default. All column vectors
-   * should normally be the default size.
-   */
-  public DoubleColumnVector() {
-    this(VectorizedRowBatch.DEFAULT_SIZE);
-  }
-
-  /**
-   * Don't use this except for testing purposes.
-   *
-   * @param len
-   */
-  public DoubleColumnVector(int len) {
-    super(len);
-    vector = new double[len];
-  }
-
-  // Copy the current object contents into the output. Only copy selected entries,
-  // as indicated by selectedInUse and the sel array.
-  public void copySelected(
-      boolean selectedInUse, int[] sel, int size, DoubleColumnVector output) {
-
-    // Output has nulls if and only if input has nulls.
-    output.noNulls = noNulls;
-    output.isRepeating = false;
-
-    // Handle repeating case
-    if (isRepeating) {
-      output.vector[0] = vector[0];
-      output.isNull[0] = isNull[0];
-      output.isRepeating = true;
-      return;
-    }
-
-    // Handle normal case
-
-    // Copy data values over
-    if (selectedInUse) {
-      for (int j = 0; j < size; j++) {
-        int i = sel[j];
-        output.vector[i] = vector[i];
-      }
-    }
-    else {
-      System.arraycopy(vector, 0, output.vector, 0, size);
-    }
-
-    // Copy nulls over if needed
-    if (!noNulls) {
-      if (selectedInUse) {
-        for (int j = 0; j < size; j++) {
-          int i = sel[j];
-          output.isNull[i] = isNull[i];
-        }
-      }
-      else {
-        System.arraycopy(isNull, 0, output.isNull, 0, size);
-      }
-    }
-  }
-
-  // Fill the column vector with the provided value
-  public void fill(double value) {
-    noNulls = true;
-    isRepeating = true;
-    vector[0] = value;
-  }
-
-  // Simplify vector by brute-force flattening noNulls and isRepeating
-  // This can be used to reduce combinatorial explosion of code paths in VectorExpressions
-  // with many arguments.
-  public void flatten(boolean selectedInUse, int[] sel, int size) {
-    flattenPush();
-    if (isRepeating) {
-      isRepeating = false;
-      double repeatVal = vector[0];
-      if (selectedInUse) {
-        for (int j = 0; j < size; j++) {
-          int i = sel[j];
-          vector[i] = repeatVal;
-        }
-      } else {
-        Arrays.fill(vector, 0, size, repeatVal);
-      }
-      flattenRepeatingNulls(selectedInUse, sel, size);
-    }
-    flattenNoNulls(selectedInUse, sel, size);
-  }
-
-  @Override
-  public void setElement(int outElementNum, int inputElementNum, ColumnVector inputVector) {
-    vector[outElementNum] = ((DoubleColumnVector) inputVector).vector[inputElementNum];
-  }
-
-  @Override
-  public void stringifyValue(StringBuilder buffer, int row) {
-    if (isRepeating) {
-      row = 0;
-    }
-    if (noNulls || !isNull[row]) {
-      buffer.append(vector[row]);
-    } else {
-      buffer.append("null");
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/9ae70cb4/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/LongColumnVector.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/LongColumnVector.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/LongColumnVector.java
deleted file mode 100644
index 5702584..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/LongColumnVector.java
+++ /dev/null
@@ -1,189 +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.
- */
-package org.apache.hadoop.hive.ql.exec.vector;
-
-import java.util.Arrays;
-
-/**
- * This class represents a nullable int column vector.
- * This class will be used for operations on all integer types (tinyint, smallint, int, bigint)
- * and as such will use a 64-bit long value to hold the biggest possible value.
- * During copy-in/copy-out, smaller int types will be converted as needed. This will
- * reduce the amount of code that needs to be generated and also will run fast since the
- * machine operates with 64-bit words.
- *
- * The vector[] field is public by design for high-performance access in the inner
- * loop of query execution.
- */
-public class LongColumnVector extends ColumnVector {
-  public long[] vector;
-  public static final long NULL_VALUE = 1;
-
-  /**
-   * Use this constructor by default. All column vectors
-   * should normally be the default size.
-   */
-  public LongColumnVector() {
-    this(VectorizedRowBatch.DEFAULT_SIZE);
-  }
-
-  /**
-   * Don't use this except for testing purposes.
-   *
-   * @param len the number of rows
-   */
-  public LongColumnVector(int len) {
-    super(len);
-    vector = new long[len];
-  }
-
-  // Copy the current object contents into the output. Only copy selected entries,
-  // as indicated by selectedInUse and the sel array.
-  public void copySelected(
-      boolean selectedInUse, int[] sel, int size, LongColumnVector output) {
-
-    // Output has nulls if and only if input has nulls.
-    output.noNulls = noNulls;
-    output.isRepeating = false;
-
-    // Handle repeating case
-    if (isRepeating) {
-      output.vector[0] = vector[0];
-      output.isNull[0] = isNull[0];
-      output.isRepeating = true;
-      return;
-    }
-
-    // Handle normal case
-
-    // Copy data values over
-    if (selectedInUse) {
-      for (int j = 0; j < size; j++) {
-        int i = sel[j];
-        output.vector[i] = vector[i];
-      }
-    }
-    else {
-      System.arraycopy(vector, 0, output.vector, 0, size);
-    }
-
-    // Copy nulls over if needed
-    if (!noNulls) {
-      if (selectedInUse) {
-        for (int j = 0; j < size; j++) {
-          int i = sel[j];
-          output.isNull[i] = isNull[i];
-        }
-      }
-      else {
-        System.arraycopy(isNull, 0, output.isNull, 0, size);
-      }
-    }
-  }
-
-  // Copy the current object contents into the output. Only copy selected entries,
-  // as indicated by selectedInUse and the sel array.
-  public void copySelected(
-      boolean selectedInUse, int[] sel, int size, DoubleColumnVector output) {
-
-    // Output has nulls if and only if input has nulls.
-    output.noNulls = noNulls;
-    output.isRepeating = false;
-
-    // Handle repeating case
-    if (isRepeating) {
-      output.vector[0] = vector[0];  // automatic conversion to double is done here
-      output.isNull[0] = isNull[0];
-      output.isRepeating = true;
-      return;
-    }
-
-    // Handle normal case
-
-    // Copy data values over
-    if (selectedInUse) {
-      for (int j = 0; j < size; j++) {
-        int i = sel[j];
-        output.vector[i] = vector[i];
-      }
-    }
-    else {
-      for(int i = 0; i < size; ++i) {
-        output.vector[i] = vector[i];
-      }
-    }
-
-    // Copy nulls over if needed
-    if (!noNulls) {
-      if (selectedInUse) {
-        for (int j = 0; j < size; j++) {
-          int i = sel[j];
-          output.isNull[i] = isNull[i];
-        }
-      }
-      else {
-        System.arraycopy(isNull, 0, output.isNull, 0, size);
-      }
-    }
-  }
-
-  // Fill the column vector with the provided value
-  public void fill(long value) {
-    noNulls = true;
-    isRepeating = true;
-    vector[0] = value;
-  }
-
-  // Simplify vector by brute-force flattening noNulls and isRepeating
-  // This can be used to reduce combinatorial explosion of code paths in VectorExpressions
-  // with many arguments.
-  public void flatten(boolean selectedInUse, int[] sel, int size) {
-    flattenPush();
-    if (isRepeating) {
-      isRepeating = false;
-      long repeatVal = vector[0];
-      if (selectedInUse) {
-        for (int j = 0; j < size; j++) {
-          int i = sel[j];
-          vector[i] = repeatVal;
-        }
-      } else {
-        Arrays.fill(vector, 0, size, repeatVal);
-      }
-      flattenRepeatingNulls(selectedInUse, sel, size);
-    }
-    flattenNoNulls(selectedInUse, sel, size);
-  }
-
-  @Override
-  public void setElement(int outElementNum, int inputElementNum, ColumnVector inputVector) {
-    vector[outElementNum] = ((LongColumnVector) inputVector).vector[inputElementNum];
-  }
-
-  @Override
-  public void stringifyValue(StringBuilder buffer, int row) {
-    if (isRepeating) {
-      row = 0;
-    }
-    if (noNulls || !isNull[row]) {
-      buffer.append(vector[row]);
-    } else {
-      buffer.append("null");
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/9ae70cb4/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatch.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatch.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatch.java
deleted file mode 100644
index 7c18da6..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatch.java
+++ /dev/null
@@ -1,186 +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.
- */
-package org.apache.hadoop.hive.ql.exec.vector;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.Writable;
-
-/**
- * A VectorizedRowBatch is a set of rows, organized with each column
- * as a vector. It is the unit of query execution, organized to minimize
- * the cost per row and achieve high cycles-per-instruction.
- * The major fields are public by design to allow fast and convenient
- * access by the vectorized query execution code.
- */
-public class VectorizedRowBatch implements Writable {
-  public int numCols;           // number of columns
-  public ColumnVector[] cols;   // a vector for each column
-  public int size;              // number of rows that qualify (i.e. haven't been filtered out)
-  public int[] selected;        // array of positions of selected values
-  public int[] projectedColumns;
-  public int projectionSize;
-
-  /*
-   * If no filtering has been applied yet, selectedInUse is false,
-   * meaning that all rows qualify. If it is true, then the selected[] array
-   * records the offsets of qualifying rows.
-   */
-  public boolean selectedInUse;
-
-  // If this is true, then there is no data in the batch -- we have hit the end of input.
-  public boolean endOfFile;
-
-  /*
-   * This number is carefully chosen to minimize overhead and typically allows
-   * one VectorizedRowBatch to fit in cache.
-   */
-  public static final int DEFAULT_SIZE = 1024;
-
-  /**
-   * Return a batch with the specified number of columns.
-   * This is the standard constructor -- all batches should be the same size
-   *
-   * @param numCols the number of columns to include in the batch
-   */
-  public VectorizedRowBatch(int numCols) {
-    this(numCols, DEFAULT_SIZE);
-  }
-
-  /**
-   * Return a batch with the specified number of columns and rows.
-   * Only call this constructor directly for testing purposes.
-   * Batch size should normally always be defaultSize.
-   *
-   * @param numCols the number of columns to include in the batch
-   * @param size  the number of rows to include in the batch
-   */
-  public VectorizedRowBatch(int numCols, int size) {
-    this.numCols = numCols;
-    this.size = size;
-    selected = new int[size];
-    selectedInUse = false;
-    this.cols = new ColumnVector[numCols];
-    projectedColumns = new int[numCols];
-
-    // Initially all columns are projected and in the same order
-    projectionSize = numCols;
-    for (int i = 0; i < numCols; i++) {
-      projectedColumns[i] = i;
-    }
-  }
-
-  /**
-   * Returns the maximum size of the batch (number of rows it can hold)
-   */
-  public int getMaxSize() {
-      return selected.length;
-  }
-
-  /**
-   * Return count of qualifying rows.
-   *
-   * @return number of rows that have not been filtered out
-   */
-  public long count() {
-    return size;
-  }
-
-  private static String toUTF8(Object o) {
-    if(o == null || o instanceof NullWritable) {
-      return "\\N"; /* as found in LazySimpleSerDe's nullSequence */
-    }
-    return o.toString();
-  }
-
-  @Override
-  public String toString() {
-    if (size == 0) {
-      return "";
-    }
-    StringBuilder b = new StringBuilder();
-    if (this.selectedInUse) {
-      for (int j = 0; j < size; j++) {
-        int i = selected[j];
-        b.append('[');
-        for (int k = 0; k < projectionSize; k++) {
-          int projIndex = projectedColumns[k];
-          ColumnVector cv = cols[projIndex];
-          if (k > 0) {
-            b.append(", ");
-          }
-          cv.stringifyValue(b, i);
-        }
-        b.append(']');
-        if (j < size - 1) {
-          b.append('\n');
-        }
-      }
-    } else {
-      for (int i = 0; i < size; i++) {
-        b.append('[');
-        for (int k = 0; k < projectionSize; k++) {
-          int projIndex = projectedColumns[k];
-          ColumnVector cv = cols[projIndex];
-          if (k > 0) {
-            b.append(", ");
-          }
-          cv.stringifyValue(b, i);
-        }
-        b.append(']');
-        if (i < size - 1) {
-          b.append('\n');
-        }
-      }
-    }
-    return b.toString();
-  }
-
-  @Override
-  public void readFields(DataInput arg0) throws IOException {
-    throw new UnsupportedOperationException("Do you really need me?");
-  }
-
-  @Override
-  public void write(DataOutput arg0) throws IOException {
-    throw new UnsupportedOperationException("Don't call me");
-  }
-
-  /**
-   * Resets the row batch to default state
-   *  - sets selectedInUse to false
-   *  - sets size to 0
-   *  - sets endOfFile to false
-   *  - resets each column
-   *  - inits each column
-   */
-  public void reset() {
-    selectedInUse = false;
-    size = 0;
-    endOfFile = false;
-    for (ColumnVector vc : cols) {
-      if (vc != null) {
-        vc.reset();
-        vc.init();
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/9ae70cb4/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentFactory.java
deleted file mode 100644
index 6ad927d..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentFactory.java
+++ /dev/null
@@ -1,39 +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.
- */
-
-package org.apache.hadoop.hive.ql.io.sarg;
-
-import com.esotericsoftware.kryo.Kryo;
-import com.esotericsoftware.kryo.io.Input;
-import org.apache.commons.codec.binary.Base64;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
-import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
-import org.apache.hadoop.hive.ql.io.sarg.SearchArgument.Builder;
-import org.apache.hadoop.hive.ql.exec.Utilities;
-import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
-import org.apache.hadoop.hive.ql.plan.TableScanDesc;
-
-/**
- * A factory for creating SearchArguments.
- */
-public class SearchArgumentFactory {
-  public static Builder newBuilder() {
-    return new SearchArgumentImpl.BuilderImpl();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/9ae70cb4/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentImpl.java
deleted file mode 100644
index 1582a75..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentImpl.java
+++ /dev/null
@@ -1,697 +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.
- */
-
-package org.apache.hadoop.hive.ql.io.sarg;
-
-import java.sql.Timestamp;
-import java.util.ArrayDeque;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Deque;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.commons.codec.binary.Base64;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import com.esotericsoftware.kryo.Kryo;
-import com.esotericsoftware.kryo.io.Output;
-
-/**
- * The implementation of SearchArguments.
- */
-final class SearchArgumentImpl implements SearchArgument {
-  public static final Log LOG = LogFactory.getLog(SearchArgumentImpl.class);
-
-  static final class PredicateLeafImpl implements PredicateLeaf {
-    private final Operator operator;
-    private final Type type;
-    private final String columnName;
-    private final Object literal;
-    private final List<Object> literalList;
-
-    // Used by kryo
-    @SuppressWarnings("unused")
-    PredicateLeafImpl() {
-      operator = null;
-      type = null;
-      columnName = null;
-      literal = null;
-      literalList = null;
-    }
-
-    PredicateLeafImpl(Operator operator,
-                      Type type,
-                      String columnName,
-                      Object literal,
-                      List<Object> literalList) {
-      this.operator = operator;
-      this.type = type;
-      this.columnName = columnName;
-      this.literal = literal;
-      if (literal != null) {
-        if (literal.getClass() != type.getValueClass()) {
-          throw new IllegalArgumentException("Wrong value class " +
-              literal.getClass().getName() + " for " + type + "." + operator +
-              " leaf");
-        }
-      }
-      this.literalList = literalList;
-      if (literalList != null) {
-        Class valueCls = type.getValueClass();
-        for(Object lit: literalList) {
-          if (lit != null && lit.getClass() != valueCls) {
-            throw new IllegalArgumentException("Wrong value class item " +
-                lit.getClass().getName() + " for " + type + "." + operator +
-                " leaf");
-          }
-        }
-      }
-    }
-
-    @Override
-    public Operator getOperator() {
-      return operator;
-    }
-
-    @Override
-    public Type getType(){
-      return type;
-    }
-
-    @Override
-    public String getColumnName() {
-      return columnName;
-    }
-
-    @Override
-    public Object getLiteral() {
-      // To get around a kryo 2.22 bug while deserialize a Timestamp into Date
-      // (https://github.com/EsotericSoftware/kryo/issues/88)
-      // When we see a Date, convert back into Timestamp
-      if (literal instanceof java.util.Date) {
-        return new Timestamp(((java.util.Date)literal).getTime());
-      }
-      return literal;
-    }
-
-    @Override
-    public List<Object> getLiteralList() {
-      return literalList;
-    }
-
-    @Override
-    public String toString() {
-      StringBuilder buffer = new StringBuilder();
-      buffer.append('(');
-      buffer.append(operator);
-      buffer.append(' ');
-      buffer.append(columnName);
-      if (literal != null) {
-        buffer.append(' ');
-        buffer.append(literal);
-      } else if (literalList != null) {
-        for(Object lit: literalList) {
-          buffer.append(' ');
-          buffer.append(lit == null ? "null" : lit.toString());
-        }
-      }
-      buffer.append(')');
-      return buffer.toString();
-    }
-
-    private static boolean isEqual(Object left, Object right) {
-
-      return left == right ||
-          (left != null && right != null && left.equals(right));
-    }
-
-    @Override
-    public boolean equals(Object other) {
-      if (other == null || other.getClass() != getClass()) {
-        return false;
-      } else if (other == this) {
-        return true;
-      } else {
-        PredicateLeafImpl o = (PredicateLeafImpl) other;
-        return operator == o.operator &&
-            type == o.type &&
-            columnName.equals(o.columnName) &&
-            isEqual(literal, o.literal) &&
-            isEqual(literalList, o.literalList);
-      }
-    }
-
-    @Override
-    public int hashCode() {
-      return operator.hashCode() +
-             type.hashCode() * 17 +
-             columnName.hashCode() * 3 * 17+
-             (literal == null ? 0 : literal.hashCode()) * 101 * 3 * 17 +
-             (literalList == null ? 0 : literalList.hashCode()) *
-                 103 * 101 * 3 * 17;
-    }
-  }
-
-
-  private final List<PredicateLeaf> leaves;
-  private final ExpressionTree expression;
-
-  SearchArgumentImpl(ExpressionTree expression, List<PredicateLeaf> leaves) {
-    this.expression = expression;
-    this.leaves = leaves;
-  }
-
-  // Used by kyro
-  @SuppressWarnings("unused")
-  SearchArgumentImpl() {
-        leaves = null;
-        expression = null;
-  }
-
-  @Override
-  public List<PredicateLeaf> getLeaves() {
-    return leaves;
-  }
-
-  @Override
-  public TruthValue evaluate(TruthValue[] leaves) {
-    return expression == null ? TruthValue.YES : expression.evaluate(leaves);
-  }
-
-  @Override
-  public ExpressionTree getExpression() {
-    return expression;
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder buffer = new StringBuilder();
-    for(int i=0; i < leaves.size(); ++i) {
-      buffer.append("leaf-");
-      buffer.append(i);
-      buffer.append(" = ");
-      buffer.append(leaves.get(i).toString());
-      buffer.append('\n');
-    }
-    buffer.append("expr = ");
-    buffer.append(expression);
-    return buffer.toString();
-  }
-
-  public String toKryo() {
-    Output out = new Output(4 * 1024, 10 * 1024 * 1024);
-    new Kryo().writeObject(out, this);
-    out.close();
-    return Base64.encodeBase64String(out.toBytes());
-  }
-
-  static class BuilderImpl implements Builder {
-
-    // max threshold for CNF conversion. having >8 elements in andList will be
-    // converted to maybe
-    private static final int CNF_COMBINATIONS_THRESHOLD = 256;
-
-    private final Deque<ExpressionTree> currentTree =
-        new ArrayDeque<ExpressionTree>();
-    private final Map<PredicateLeaf, Integer> leaves =
-        new HashMap<PredicateLeaf, Integer>();
-    private final ExpressionTree root =
-        new ExpressionTree(ExpressionTree.Operator.AND);
-    {
-      currentTree.add(root);
-    }
-
-    @Override
-    public Builder startOr() {
-      ExpressionTree node = new ExpressionTree(ExpressionTree.Operator.OR);
-      currentTree.getFirst().getChildren().add(node);
-      currentTree.addFirst(node);
-      return this;
-    }
-
-    @Override
-    public Builder startAnd() {
-      ExpressionTree node = new ExpressionTree(ExpressionTree.Operator.AND);
-      currentTree.getFirst().getChildren().add(node);
-      currentTree.addFirst(node);
-      return this;
-    }
-
-    @Override
-    public Builder startNot() {
-      ExpressionTree node = new ExpressionTree(ExpressionTree.Operator.NOT);
-      currentTree.getFirst().getChildren().add(node);
-      currentTree.addFirst(node);
-      return this;
-    }
-
-    @Override
-    public Builder end() {
-      ExpressionTree current = currentTree.removeFirst();
-      if (current.getChildren().size() == 0) {
-        throw new IllegalArgumentException("Can't create expression " + root +
-            " with no children.");
-      }
-      if (current.getOperator() == ExpressionTree.Operator.NOT &&
-          current.getChildren().size() != 1) {
-        throw new IllegalArgumentException("Can't create not expression " +
-            current + " with more than 1 child.");
-      }
-      return this;
-    }
-
-    private int addLeaf(PredicateLeaf leaf) {
-      Integer result = leaves.get(leaf);
-      if (result == null) {
-        int id = leaves.size();
-        leaves.put(leaf, id);
-        return id;
-      } else {
-        return result;
-      }
-    }
-
-    @Override
-    public Builder lessThan(String column, PredicateLeaf.Type type,
-                            Object literal) {
-      ExpressionTree parent = currentTree.getFirst();
-      if (column == null || literal == null) {
-        parent.getChildren().add(new ExpressionTree(TruthValue.YES_NO_NULL));
-      } else {
-        PredicateLeaf leaf =
-            new PredicateLeafImpl(PredicateLeaf.Operator.LESS_THAN,
-                type, column, literal, null);
-        parent.getChildren().add(new ExpressionTree(addLeaf(leaf)));
-      }
-      return this;
-    }
-
-    @Override
-    public Builder lessThanEquals(String column, PredicateLeaf.Type type,
-                                  Object literal) {
-      ExpressionTree parent = currentTree.getFirst();
-      if (column == null || literal == null) {
-        parent.getChildren().add(new ExpressionTree(TruthValue.YES_NO_NULL));
-      } else {
-        PredicateLeaf leaf =
-            new PredicateLeafImpl(PredicateLeaf.Operator.LESS_THAN_EQUALS,
-                type, column, literal, null);
-        parent.getChildren().add(new ExpressionTree(addLeaf(leaf)));
-      }
-      return this;
-    }
-
-    @Override
-    public Builder equals(String column, PredicateLeaf.Type type,
-                          Object literal) {
-      ExpressionTree parent = currentTree.getFirst();
-      if (column == null || literal == null) {
-        parent.getChildren().add(new ExpressionTree(TruthValue.YES_NO_NULL));
-      } else {
-        PredicateLeaf leaf =
-            new PredicateLeafImpl(PredicateLeaf.Operator.EQUALS,
-                type, column, literal, null);
-        parent.getChildren().add(new ExpressionTree(addLeaf(leaf)));
-      }
-      return this;
-    }
-
-    @Override
-    public Builder nullSafeEquals(String column, PredicateLeaf.Type type,
-                                  Object literal) {
-      ExpressionTree parent = currentTree.getFirst();
-      if (column == null || literal == null) {
-        parent.getChildren().add(new ExpressionTree(TruthValue.YES_NO_NULL));
-      } else {
-        PredicateLeaf leaf =
-            new PredicateLeafImpl(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-                type, column, literal, null);
-        parent.getChildren().add(new ExpressionTree(addLeaf(leaf)));
-      }
-      return this;
-    }
-
-    @Override
-    public Builder in(String column, PredicateLeaf.Type type,
-                      Object... literal) {
-      ExpressionTree parent = currentTree.getFirst();
-      if (column  == null || literal == null) {
-        parent.getChildren().add(new ExpressionTree(TruthValue.YES_NO_NULL));
-      } else {
-        if (literal.length == 0) {
-          throw new IllegalArgumentException("Can't create in expression with "
-              + "no arguments");
-        }
-        List<Object> argList = new ArrayList<Object>();
-        argList.addAll(Arrays.asList(literal));
-
-        PredicateLeaf leaf =
-            new PredicateLeafImpl(PredicateLeaf.Operator.IN,
-                type, column, null, argList);
-        parent.getChildren().add(new ExpressionTree(addLeaf(leaf)));
-      }
-      return this;
-    }
-
-    @Override
-    public Builder isNull(String column, PredicateLeaf.Type type) {
-      ExpressionTree parent = currentTree.getFirst();
-      if (column == null) {
-        parent.getChildren().add(new ExpressionTree(TruthValue.YES_NO_NULL));
-      } else {
-        PredicateLeaf leaf =
-            new PredicateLeafImpl(PredicateLeaf.Operator.IS_NULL,
-                type, column, null, null);
-        parent.getChildren().add(new ExpressionTree(addLeaf(leaf)));
-      }
-      return this;
-    }
-
-    @Override
-    public Builder between(String column, PredicateLeaf.Type type, Object lower,
-                           Object upper) {
-      ExpressionTree parent = currentTree.getFirst();
-      if (column == null || lower == null || upper == null) {
-        parent.getChildren().add(new ExpressionTree(TruthValue.YES_NO_NULL));
-      } else {
-        List<Object> argList = new ArrayList<Object>();
-        argList.add(lower);
-        argList.add(upper);
-        PredicateLeaf leaf =
-            new PredicateLeafImpl(PredicateLeaf.Operator.BETWEEN,
-                type, column, null, argList);
-        parent.getChildren().add(new ExpressionTree(addLeaf(leaf)));
-      }
-      return this;
-    }
-
-    @Override
-    public Builder literal(TruthValue truth) {
-      ExpressionTree parent = currentTree.getFirst();
-      parent.getChildren().add(new ExpressionTree(truth));
-      return this;
-    }
-
-    /**
-     * Recursively explore the tree to find the leaves that are still reachable
-     * after optimizations.
-     * @param tree the node to check next
-     * @param next the next available leaf id
-     * @param leafReorder
-     * @return the next available leaf id
-     */
-    static int compactLeaves(ExpressionTree tree, int next, int[] leafReorder) {
-      if (tree.getOperator() == ExpressionTree.Operator.LEAF) {
-        int oldLeaf = tree.getLeaf();
-        if (leafReorder[oldLeaf] == -1) {
-          leafReorder[oldLeaf] = next++;
-        }
-      } else if (tree.getChildren() != null){
-        for(ExpressionTree child: tree.getChildren()) {
-          next = compactLeaves(child, next, leafReorder);
-        }
-      }
-      return next;
-    }
-
-    /**
-     * Rewrite expression tree to update the leaves.
-     * @param root the root of the tree to fix
-     * @param leafReorder a map from old leaf ids to new leaf ids
-     * @return the fixed root
-     */
-    static ExpressionTree rewriteLeaves(ExpressionTree root,
-                                        int[] leafReorder) {
-      if (root.getOperator() == ExpressionTree.Operator.LEAF) {
-        return new ExpressionTree(leafReorder[root.getLeaf()]);
-      } else if (root.getChildren() != null){
-        List<ExpressionTree> children = root.getChildren();
-        for(int i=0; i < children.size(); ++i) {
-          children.set(i, rewriteLeaves(children.get(i), leafReorder));
-        }
-      }
-      return root;
-    }
-
-    @Override
-    public SearchArgument build() {
-      if (currentTree.size() != 1) {
-        throw new IllegalArgumentException("Failed to end " +
-            currentTree.size() + " operations.");
-      }
-      ExpressionTree optimized = pushDownNot(root);
-      optimized = foldMaybe(optimized);
-      optimized = flatten(optimized);
-      optimized = convertToCNF(optimized);
-      optimized = flatten(optimized);
-      int leafReorder[] = new int[leaves.size()];
-      Arrays.fill(leafReorder, -1);
-      int newLeafCount = compactLeaves(optimized, 0, leafReorder);
-      optimized = rewriteLeaves(optimized, leafReorder);
-      ArrayList<PredicateLeaf> leafList = new ArrayList<>(newLeafCount);
-      // expand list to correct size
-      for(int i=0; i < newLeafCount; ++i) {
-        leafList.add(null);
-      }
-      // build the new list
-      for(Map.Entry<PredicateLeaf, Integer> elem: leaves.entrySet()) {
-        int newLoc = leafReorder[elem.getValue()];
-        if (newLoc != -1) {
-          leafList.set(newLoc, elem.getKey());
-        }
-      }
-      return new SearchArgumentImpl(optimized, leafList);
-    }
-
-    /**
-     * Push the negations all the way to just before the leaves. Also remove
-     * double negatives.
-     * @param root the expression to normalize
-     * @return the normalized expression, which may share some or all of the
-     * nodes of the original expression.
-     */
-    static ExpressionTree pushDownNot(ExpressionTree root) {
-      if (root.getOperator() == ExpressionTree.Operator.NOT) {
-        ExpressionTree child = root.getChildren().get(0);
-        switch (child.getOperator()) {
-          case NOT:
-            return pushDownNot(child.getChildren().get(0));
-          case CONSTANT:
-            return  new ExpressionTree(child.getConstant().not());
-          case AND:
-            root = new ExpressionTree(ExpressionTree.Operator.OR);
-            for(ExpressionTree kid: child.getChildren()) {
-              root.getChildren().add(pushDownNot(new
-                  ExpressionTree(ExpressionTree.Operator.NOT, kid)));
-            }
-            break;
-          case OR:
-            root = new ExpressionTree(ExpressionTree.Operator.AND);
-            for(ExpressionTree kid: child.getChildren()) {
-              root.getChildren().add(pushDownNot(new ExpressionTree
-                  (ExpressionTree.Operator.NOT, kid)));
-            }
-            break;
-          // for leaf, we don't do anything
-          default:
-            break;
-        }
-      } else if (root.getChildren() != null) {
-        // iterate through children and push down not for each one
-        for(int i=0; i < root.getChildren().size(); ++i) {
-          root.getChildren().set(i, pushDownNot(root.getChildren().get(i)));
-        }
-      }
-      return root;
-    }
-
-    /**
-     * Remove MAYBE values from the expression. If they are in an AND operator,
-     * they are dropped. If they are in an OR operator, they kill their parent.
-     * This assumes that pushDownNot has already been called.
-     * @param expr The expression to clean up
-     * @return The cleaned up expression
-     */
-    static ExpressionTree foldMaybe(ExpressionTree expr) {
-      if (expr.getChildren() != null) {
-        for(int i=0; i < expr.getChildren().size(); ++i) {
-          ExpressionTree child = foldMaybe(expr.getChildren().get(i));
-          if (child.getConstant() == TruthValue.YES_NO_NULL) {
-            switch (expr.getOperator()) {
-              case AND:
-                expr.getChildren().remove(i);
-                i -= 1;
-                break;
-              case OR:
-                // a maybe will kill the or condition
-                return child;
-              default:
-                throw new IllegalStateException("Got a maybe as child of " +
-                    expr);
-            }
-          } else {
-            expr.getChildren().set(i, child);
-          }
-        }
-        if (expr.getChildren().isEmpty()) {
-          return new ExpressionTree(TruthValue.YES_NO_NULL);
-        }
-      }
-      return expr;
-    }
-
-    /**
-     * Converts multi-level ands and ors into single level ones.
-     * @param root the expression to flatten
-     * @return the flattened expression, which will always be root with
-     *   potentially modified children.
-     */
-    static ExpressionTree flatten(ExpressionTree root) {
-      if (root.getChildren() != null) {
-        // iterate through the index, so that if we add more children,
-        // they don't get re-visited
-        for(int i=0; i < root.getChildren().size(); ++i) {
-          ExpressionTree child = flatten(root.getChildren().get(i));
-          // do we need to flatten?
-          if (child.getOperator() == root.getOperator() &&
-              child.getOperator() != ExpressionTree.Operator.NOT) {
-            boolean first = true;
-            for(ExpressionTree grandkid: child.getChildren()) {
-              // for the first grandkid replace the original parent
-              if (first) {
-                first = false;
-                root.getChildren().set(i, grandkid);
-              } else {
-                root.getChildren().add(++i, grandkid);
-              }
-            }
-          } else {
-            root.getChildren().set(i, child);
-          }
-        }
-        // if we have a singleton AND or OR, just return the child
-        if ((root.getOperator() == ExpressionTree.Operator.OR ||
-            root.getOperator() == ExpressionTree.Operator.AND) &&
-            root.getChildren().size() == 1) {
-          return root.getChildren().get(0);
-        }
-      }
-      return root;
-    }
-
-    /**
-     * Generate all combinations of items on the andList. For each item on the
-     * andList, it generates all combinations of one child from each and
-     * expression. Thus, (and a b) (and c d) will be expanded to: (or a c)
-     * (or a d) (or b c) (or b d). If there are items on the nonAndList, they
-     * are added to each or expression.
-     * @param result a list to put the results onto
-     * @param andList a list of and expressions
-     * @param nonAndList a list of non-and expressions
-     */
-    private static void generateAllCombinations(List<ExpressionTree> result,
-                                                List<ExpressionTree> andList,
-                                                List<ExpressionTree> nonAndList
-    ) {
-      List<ExpressionTree> kids = andList.get(0).getChildren();
-      if (result.isEmpty()) {
-        for(ExpressionTree kid: kids) {
-          ExpressionTree or = new ExpressionTree(ExpressionTree.Operator.OR);
-          result.add(or);
-          for(ExpressionTree node: nonAndList) {
-            or.getChildren().add(new ExpressionTree(node));
-          }
-          or.getChildren().add(kid);
-        }
-      } else {
-        List<ExpressionTree> work = new ArrayList<ExpressionTree>(result);
-        result.clear();
-        for(ExpressionTree kid: kids) {
-          for(ExpressionTree or: work) {
-            ExpressionTree copy = new ExpressionTree(or);
-            copy.getChildren().add(kid);
-            result.add(copy);
-          }
-        }
-      }
-      if (andList.size() > 1) {
-        generateAllCombinations(result, andList.subList(1, andList.size()),
-            nonAndList);
-      }
-    }
-
-    /**
-     * Convert an expression so that the top level operator is AND with OR
-     * operators under it. This routine assumes that all of the NOT operators
-     * have been pushed to the leaves via pushdDownNot.
-     * @param root the expression
-     * @return the normalized expression
-     */
-    static ExpressionTree convertToCNF(ExpressionTree root) {
-      if (root.getChildren() != null) {
-        // convert all of the children to CNF
-        int size = root.getChildren().size();
-        for(int i=0; i < size; ++i) {
-          root.getChildren().set(i, convertToCNF(root.getChildren().get(i)));
-        }
-        if (root.getOperator() == ExpressionTree.Operator.OR) {
-          // a list of leaves that weren't under AND expressions
-          List<ExpressionTree> nonAndList = new ArrayList<ExpressionTree>();
-          // a list of AND expressions that we need to distribute
-          List<ExpressionTree> andList = new ArrayList<ExpressionTree>();
-          for(ExpressionTree child: root.getChildren()) {
-            if (child.getOperator() == ExpressionTree.Operator.AND) {
-              andList.add(child);
-            } else if (child.getOperator() == ExpressionTree.Operator.OR) {
-              // pull apart the kids of the OR expression
-              for(ExpressionTree grandkid: child.getChildren()) {
-                nonAndList.add(grandkid);
-              }
-            } else {
-              nonAndList.add(child);
-            }
-          }
-          if (!andList.isEmpty()) {
-            if (checkCombinationsThreshold(andList)) {
-              root = new ExpressionTree(ExpressionTree.Operator.AND);
-              generateAllCombinations(root.getChildren(), andList, nonAndList);
-            } else {
-              root = new ExpressionTree(TruthValue.YES_NO_NULL);
-            }
-          }
-        }
-      }
-      return root;
-    }
-
-    private static boolean checkCombinationsThreshold(List<ExpressionTree> andList) {
-      int numComb = 1;
-      for (ExpressionTree tree : andList) {
-        numComb *= tree.getChildren().size();
-        if (numComb > CNF_COMBINATIONS_THRESHOLD) {
-          return false;
-        }
-      }
-      return true;
-    }
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/9ae70cb4/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
index 46deda5..6cb8529 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
@@ -17,6 +17,9 @@
  */
 package org.apache.hadoop.hive.ql.io.orc;
 
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.io.Output;
+
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
@@ -40,6 +43,7 @@ import java.util.Set;
 import java.util.TimeZone;
 import java.util.TreeSet;
 
+import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -106,6 +110,13 @@ import org.junit.rules.TestName;
 
 public class TestInputOutputFormat {
 
+  public static String toKryo(SearchArgument sarg) {
+    Output out = new Output(4 * 1024, 10 * 1024 * 1024);
+    new Kryo().writeObject(out, sarg);
+    out.close();
+    return Base64.encodeBase64String(out.toBytes());
+  }
+
   Path workDir = new Path(System.getProperty("test.tmp.dir","target/tmp"));
   static final int MILLIS_IN_DAY = 1000 * 60 * 60 * 24;
   private static final SimpleDateFormat DATE_FORMAT =
@@ -1751,7 +1762,7 @@ public class TestInputOutputFormat {
     types.add(builder.build());
     SearchArgument isNull = SearchArgumentFactory.newBuilder()
         .startAnd().isNull("cost", PredicateLeaf.Type.INTEGER).end().build();
-    conf.set(ConvertAstToSearchArg.SARG_PUSHDOWN, isNull.toKryo());
+    conf.set(ConvertAstToSearchArg.SARG_PUSHDOWN, toKryo(isNull));
     conf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR,
         "url,cost");
     options.include(new boolean[]{true, true, false, true, false});
@@ -1798,7 +1809,7 @@ public class TestInputOutputFormat {
             .lessThan("z", PredicateLeaf.Type.INTEGER, new Integer(0))
             .end()
             .build();
-    conf.set("sarg.pushdown", sarg.toKryo());
+    conf.set("sarg.pushdown", toKryo(sarg));
     conf.set("hive.io.file.readcolumn.names", "z,r");
     properties.setProperty("columns", "z,r");
     properties.setProperty("columns.types", "int:struct<x:int,y:int>");
@@ -1840,7 +1851,7 @@ public class TestInputOutputFormat {
             .lessThan("z", PredicateLeaf.Type.STRING, new String("foo"))
             .end()
             .build();
-    conf.set("sarg.pushdown", sarg.toKryo());
+    conf.set("sarg.pushdown", toKryo(sarg));
     conf.set("hive.io.file.readcolumn.names", "z");
     properties.setProperty("columns", "z");
     properties.setProperty("columns.types", "string");

http://git-wip-us.apache.org/repos/asf/hive/blob/9ae70cb4/ql/src/test/org/apache/hadoop/hive/ql/io/sarg/TestSearchArgumentImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/sarg/TestSearchArgumentImpl.java b/ql/src/test/org/apache/hadoop/hive/ql/io/sarg/TestSearchArgumentImpl.java
index 3a51f4a..20de846 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/sarg/TestSearchArgumentImpl.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/sarg/TestSearchArgumentImpl.java
@@ -25,6 +25,7 @@ import com.google.common.collect.Sets;
 
 import org.apache.hadoop.hive.common.type.HiveChar;
 import org.apache.hadoop.hive.common.type.HiveVarchar;
+import org.apache.hadoop.hive.ql.io.orc.TestInputOutputFormat;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument.TruthValue;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentImpl.PredicateLeafImpl;
 import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
@@ -464,7 +465,7 @@ public class TestSearchArgumentImpl {
         .end()
         .build();
 
-    String serializedSarg = sarg.toKryo();
+    String serializedSarg = TestInputOutputFormat.toKryo(sarg);
     SearchArgument sarg2 = ConvertAstToSearchArg.create(serializedSarg);
 
     Field literalField = PredicateLeafImpl.class.getDeclaredField("literal");

http://git-wip-us.apache.org/repos/asf/hive/blob/9ae70cb4/serde/src/java/org/apache/hadoop/hive/ql/io/sarg/ExpressionTree.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/ql/io/sarg/ExpressionTree.java b/serde/src/java/org/apache/hadoop/hive/ql/io/sarg/ExpressionTree.java
deleted file mode 100644
index 2dd3a45..0000000
--- a/serde/src/java/org/apache/hadoop/hive/ql/io/sarg/ExpressionTree.java
+++ /dev/null
@@ -1,157 +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.
- */
-
-package org.apache.hadoop.hive.ql.io.sarg;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-/**
- * The inner representation of the SearchArgument. Most users should not
- * need this interface, it is only for file formats that need to translate
- * the SearchArgument into an internal form.
- */
-public class ExpressionTree {
-  public enum Operator {OR, AND, NOT, LEAF, CONSTANT}
-  private final Operator operator;
-  private final List<ExpressionTree> children;
-  private final int leaf;
-  private final SearchArgument.TruthValue constant;
-
-  ExpressionTree() {
-    operator = null;
-    children = null;
-    leaf = 0;
-    constant = null;
-  }
-
-  ExpressionTree(Operator op, ExpressionTree... kids) {
-    operator = op;
-    children = new ArrayList<ExpressionTree>();
-    leaf = -1;
-    this.constant = null;
-    Collections.addAll(children, kids);
-  }
-
-  ExpressionTree(int leaf) {
-    operator = Operator.LEAF;
-    children = null;
-    this.leaf = leaf;
-    this.constant = null;
-  }
-
-  ExpressionTree(SearchArgument.TruthValue constant) {
-    operator = Operator.CONSTANT;
-    children = null;
-    this.leaf = -1;
-    this.constant = constant;
-  }
-
-  ExpressionTree(ExpressionTree other) {
-    this.operator = other.operator;
-    if (other.children == null) {
-      this.children = null;
-    } else {
-      this.children = new ArrayList<ExpressionTree>();
-      for(ExpressionTree child: other.children) {
-        children.add(new ExpressionTree(child));
-      }
-    }
-    this.leaf = other.leaf;
-    this.constant = other.constant;
-  }
-
-  public SearchArgument.TruthValue evaluate(SearchArgument.TruthValue[] leaves
-                                            ) {
-    SearchArgument.TruthValue result = null;
-    switch (operator) {
-      case OR:
-        for(ExpressionTree child: children) {
-          result = child.evaluate(leaves).or(result);
-        }
-        return result;
-      case AND:
-        for(ExpressionTree child: children) {
-          result = child.evaluate(leaves).and(result);
-        }
-        return result;
-      case NOT:
-        return children.get(0).evaluate(leaves).not();
-      case LEAF:
-        return leaves[leaf];
-      case CONSTANT:
-        return constant;
-      default:
-        throw new IllegalStateException("Unknown operator: " + operator);
-    }
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder buffer = new StringBuilder();
-    switch (operator) {
-      case OR:
-        buffer.append("(or");
-        for(ExpressionTree child: children) {
-          buffer.append(' ');
-          buffer.append(child.toString());
-        }
-        buffer.append(')');
-        break;
-      case AND:
-        buffer.append("(and");
-        for(ExpressionTree child: children) {
-          buffer.append(' ');
-          buffer.append(child.toString());
-        }
-        buffer.append(')');
-        break;
-      case NOT:
-        buffer.append("(not ");
-        buffer.append(children.get(0));
-        buffer.append(')');
-        break;
-      case LEAF:
-        buffer.append("leaf-");
-        buffer.append(leaf);
-        break;
-      case CONSTANT:
-        buffer.append(constant);
-        break;
-    }
-    return buffer.toString();
-  }
-
-  public Operator getOperator() {
-    return operator;
-  }
-
-  public List<ExpressionTree> getChildren() {
-    return children;
-  }
-
-  public SearchArgument.TruthValue getConstant() {
-    return constant;
-  }
-
-  public int getLeaf() {
-    return leaf;
-  }
-}
-


[42/43] hive git commit: HIVE-11258 - The function drop_database_core() of HiveMetaStore.java may not drop all the tables (Aihua Xu, reviewed by Chao Sun)

Posted by xu...@apache.org.
HIVE-11258 - The function drop_database_core() of HiveMetaStore.java may not drop all the tables (Aihua Xu, reviewed by Chao Sun)


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

Branch: refs/heads/spark
Commit: 7df9d7a93c7e2dce3fe81213a61fe21d11a7bdb9
Parents: bfe8591
Author: Aihua Xu <ai...@gmail.com>
Authored: Thu Jul 30 09:38:31 2015 -0700
Committer: Chao Sun <su...@apache.org>
Committed: Thu Jul 30 09:38:39 2015 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/metastore/HiveMetaStore.java  | 11 ++++-------
 1 file changed, 4 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/7df9d7a9/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index ee2cea0..72a837c 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -1039,14 +1039,9 @@ public class HiveMetaStore extends ThriftHiveMetastore {
             ConfVars.METASTORE_BATCH_RETRIEVE_MAX);
 
         int startIndex = 0;
-        int endIndex = -1;
         // retrieve the tables from the metastore in batches to alleviate memory constraints
-        while (endIndex < allTables.size() - 1) {
-          startIndex = endIndex + 1;
-          endIndex = endIndex + tableBatchSize;
-          if (endIndex >= allTables.size()) {
-            endIndex = allTables.size() - 1;
-          }
+        while (startIndex < allTables.size()) {
+          int endIndex = Math.min(startIndex + tableBatchSize, allTables.size());
 
           List<Table> tables = null;
           try {
@@ -1082,6 +1077,8 @@ public class HiveMetaStore extends ThriftHiveMetastore {
               // Drop the table but not its data
               drop_table(name, table.getTableName(), false);
             }
+
+            startIndex = endIndex;
           }
         }
 


[17/43] hive git commit: HIVE-11301 : thrift metastore issue when getting stats results in disconnect (Pengcheng Xiong, reviewed by Sergey Shelukhin)

Posted by xu...@apache.org.
HIVE-11301 : thrift metastore issue when getting stats results in disconnect (Pengcheng Xiong, reviewed by Sergey Shelukhin)


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

Branch: refs/heads/spark
Commit: 393d690bb2e61331f65cf1de4826c0248911740c
Parents: e57c360
Author: Sergey Shelukhin <se...@apache.org>
Authored: Thu Jul 23 10:35:57 2015 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Thu Jul 23 10:35:57 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hive/ql/stats/StatsUtils.java |  10 +-
 .../test/queries/clientpositive/stats_ppr_all.q |  24 ++
 .../results/clientpositive/stats_ppr_all.q.out  | 284 +++++++++++++++++++
 3 files changed, 316 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/393d690b/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
index 42570ce..a069394 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsUtils.java
@@ -249,8 +249,14 @@ public class StatsUtils {
           partNames.add(part.getName());
         }
         neededColumns = processNeededColumns(schema, neededColumns);
-        AggrStats aggrStats = Hive.get().getAggrColStatsFor(table.getDbName(), table.getTableName(),
-            neededColumns, partNames);
+        AggrStats aggrStats = null;
+        // We check the sizes of neededColumns and partNames here. If either
+        // size is 0, aggrStats is null after several retries. Thus, we can
+        // skip the step to connect to the metastore.
+        if (neededColumns.size() > 0 && partNames.size() > 0) {
+          aggrStats = Hive.get().getAggrColStatsFor(table.getDbName(), table.getTableName(),
+              neededColumns, partNames);
+        }
         if (null == aggrStats || null == aggrStats.getColStats()
             || aggrStats.getColStatsSize() == 0) {
           // There are some partitions with no state (or we didn't fetch any state).

http://git-wip-us.apache.org/repos/asf/hive/blob/393d690b/ql/src/test/queries/clientpositive/stats_ppr_all.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/stats_ppr_all.q b/ql/src/test/queries/clientpositive/stats_ppr_all.q
new file mode 100644
index 0000000..b611e83
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/stats_ppr_all.q
@@ -0,0 +1,24 @@
+set hive.stats.fetch.column.stats=true;
+
+drop table ss;
+
+CREATE TABLE ss (
+    sales_order_id  BIGINT,
+    order_amount    FLOAT)
+PARTITIONED BY (country STRING, year INT, month INT, day INT) stored as orc;
+
+insert into ss partition(country="US", year=2015, month=1, day=1) values(1,22.0);
+insert into ss partition(country="US", year=2015, month=2, day=1) values(2,2.0);
+insert into ss partition(country="US", year=2015, month=1, day=2) values(1,2.0);
+
+ANALYZE TABLE ss PARTITION(country,year,month,day) compute statistics for columns;
+
+explain select sum(order_amount) from ss where (country="US" and year=2015 and month=2 and day=1);
+
+explain select sum(order_amount) from ss where (year*10000+month*100+day) = "2015010" and 1>0;
+
+explain select sum(order_amount) from ss where (year*100+month*10+day) = "201511" and 1>0;
+
+explain select sum(order_amount) from ss where (year*100+month*10+day) > "201511" and 1>0;
+
+explain select '1' from ss where (year*100+month*10+day) > "201511";
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/393d690b/ql/src/test/results/clientpositive/stats_ppr_all.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/stats_ppr_all.q.out b/ql/src/test/results/clientpositive/stats_ppr_all.q.out
new file mode 100644
index 0000000..5f6f5d4
--- /dev/null
+++ b/ql/src/test/results/clientpositive/stats_ppr_all.q.out
@@ -0,0 +1,284 @@
+PREHOOK: query: drop table ss
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table ss
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: CREATE TABLE ss (
+    sales_order_id  BIGINT,
+    order_amount    FLOAT)
+PARTITIONED BY (country STRING, year INT, month INT, day INT) stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ss
+POSTHOOK: query: CREATE TABLE ss (
+    sales_order_id  BIGINT,
+    order_amount    FLOAT)
+PARTITIONED BY (country STRING, year INT, month INT, day INT) stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ss
+PREHOOK: query: insert into ss partition(country="US", year=2015, month=1, day=1) values(1,22.0)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__1
+PREHOOK: Output: default@ss@country=US/year=2015/month=1/day=1
+POSTHOOK: query: insert into ss partition(country="US", year=2015, month=1, day=1) values(1,22.0)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__1
+POSTHOOK: Output: default@ss@country=US/year=2015/month=1/day=1
+POSTHOOK: Lineage: ss PARTITION(country=US,year=2015,month=1,day=1).order_amount EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+POSTHOOK: Lineage: ss PARTITION(country=US,year=2015,month=1,day=1).sales_order_id EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+PREHOOK: query: insert into ss partition(country="US", year=2015, month=2, day=1) values(2,2.0)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__2
+PREHOOK: Output: default@ss@country=US/year=2015/month=2/day=1
+POSTHOOK: query: insert into ss partition(country="US", year=2015, month=2, day=1) values(2,2.0)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__2
+POSTHOOK: Output: default@ss@country=US/year=2015/month=2/day=1
+POSTHOOK: Lineage: ss PARTITION(country=US,year=2015,month=2,day=1).order_amount EXPRESSION [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+POSTHOOK: Lineage: ss PARTITION(country=US,year=2015,month=2,day=1).sales_order_id EXPRESSION [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+PREHOOK: query: insert into ss partition(country="US", year=2015, month=1, day=2) values(1,2.0)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__3
+PREHOOK: Output: default@ss@country=US/year=2015/month=1/day=2
+POSTHOOK: query: insert into ss partition(country="US", year=2015, month=1, day=2) values(1,2.0)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__3
+POSTHOOK: Output: default@ss@country=US/year=2015/month=1/day=2
+POSTHOOK: Lineage: ss PARTITION(country=US,year=2015,month=1,day=2).order_amount EXPRESSION [(values__tmp__table__3)values__tmp__table__3.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+POSTHOOK: Lineage: ss PARTITION(country=US,year=2015,month=1,day=2).sales_order_id EXPRESSION [(values__tmp__table__3)values__tmp__table__3.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+PREHOOK: query: ANALYZE TABLE ss PARTITION(country,year,month,day) compute statistics for columns
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ss
+PREHOOK: Input: default@ss@country=US/year=2015/month=1/day=1
+PREHOOK: Input: default@ss@country=US/year=2015/month=1/day=2
+PREHOOK: Input: default@ss@country=US/year=2015/month=2/day=1
+#### A masked pattern was here ####
+POSTHOOK: query: ANALYZE TABLE ss PARTITION(country,year,month,day) compute statistics for columns
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ss
+POSTHOOK: Input: default@ss@country=US/year=2015/month=1/day=1
+POSTHOOK: Input: default@ss@country=US/year=2015/month=1/day=2
+POSTHOOK: Input: default@ss@country=US/year=2015/month=2/day=1
+#### A masked pattern was here ####
+PREHOOK: query: explain select sum(order_amount) from ss where (country="US" and year=2015 and month=2 and day=1)
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select sum(order_amount) from ss where (country="US" and year=2015 and month=2 and day=1)
+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: ss
+            Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+            Select Operator
+              expressions: order_amount (type: float)
+              outputColumnNames: _col0
+              Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+              Group By Operator
+                aggregations: sum(_col0)
+                mode: hash
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: double)
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: sum(VALUE._col0)
+          mode: mergepartial
+          outputColumnNames: _col0
+          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+            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 sum(order_amount) from ss where (year*10000+month*100+day) = "2015010" and 1>0
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select sum(order_amount) from ss where (year*10000+month*100+day) = "2015010" and 1>0
+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: ss
+            Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: PARTIAL
+            Filter Operator
+              predicate: (UDFToDouble((((year * 10000) + (month * 100)) + day)) = 2015010.0) (type: boolean)
+              Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: PARTIAL
+              Select Operator
+                expressions: order_amount (type: float)
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: PARTIAL
+                Group By Operator
+                  aggregations: sum(_col0)
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: PARTIAL
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: PARTIAL
+                    value expressions: _col0 (type: double)
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: sum(VALUE._col0)
+          mode: mergepartial
+          outputColumnNames: _col0
+          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: PARTIAL
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: PARTIAL
+            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 sum(order_amount) from ss where (year*100+month*10+day) = "201511" and 1>0
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select sum(order_amount) from ss where (year*100+month*10+day) = "201511" and 1>0
+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: ss
+            Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+            Select Operator
+              expressions: order_amount (type: float)
+              outputColumnNames: _col0
+              Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+              Group By Operator
+                aggregations: sum(_col0)
+                mode: hash
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  sort order: 
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: _col0 (type: double)
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: sum(VALUE._col0)
+          mode: mergepartial
+          outputColumnNames: _col0
+          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+            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 sum(order_amount) from ss where (year*100+month*10+day) > "201511" and 1>0
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select sum(order_amount) from ss where (year*100+month*10+day) > "201511" and 1>0
+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: ss
+            Statistics: Num rows: 2 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+            Filter Operator
+              predicate: (UDFToDouble(((201500 + (month * 10)) + day)) > 201511.0) (type: boolean)
+              Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: order_amount (type: float)
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
+                Group By Operator
+                  aggregations: sum(_col0)
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: double)
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: sum(VALUE._col0)
+          mode: mergepartial
+          outputColumnNames: _col0
+          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+            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 '1' from ss where (year*100+month*10+day) > "201511"
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select '1' from ss where (year*100+month*10+day) > "201511"
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        TableScan
+          alias: ss
+          Statistics: Num rows: 2 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
+          Filter Operator
+            predicate: (UDFToDouble(((201500 + (month * 10)) + day)) > 201511.0) (type: boolean)
+            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+            Select Operator
+              expressions: '1' (type: string)
+              outputColumnNames: _col0
+              Statistics: Num rows: 1 Data size: 85 Basic stats: COMPLETE Column stats: COMPLETE
+              ListSink
+


[40/43] hive git commit: HIVE-11214: Insert into ACID table switches vectorization off (Matt McCline, reviewed by Eugene Koifman)

Posted by xu...@apache.org.
HIVE-11214: Insert into ACID table switches vectorization off (Matt McCline, reviewed by Eugene Koifman)


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

Branch: refs/heads/spark
Commit: 0ec5185a28177a04cf4f59a38b7c93da3b6e4565
Parents: 4ee17e5
Author: Matt McCline <mm...@hortonworks.com>
Authored: Wed Jul 29 15:03:20 2015 -0700
Committer: Matt McCline <mm...@hortonworks.com>
Committed: Wed Jul 29 15:03:52 2015 -0700

----------------------------------------------------------------------
 .../test/resources/testconfiguration.properties |  1 +
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |  4 ---
 .../test/queries/clientpositive/vector_acid3.q  | 17 +++++++++++
 .../clientpositive/tez/vector_acid3.q.out       | 31 ++++++++++++++++++++
 .../results/clientpositive/vector_acid3.q.out   | 31 ++++++++++++++++++++
 5 files changed, 80 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/0ec5185a/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index fbde465..aabf6fc 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -186,6 +186,7 @@ minitez.query.files.shared=alter_merge_2_orc.q,\
   update_where_non_partitioned.q,\
   update_where_partitioned.q,\
   update_two_cols.q,\
+  vector_acid3.q,\
   vector_aggregate_9.q,\
   vector_between_in.q,\
   vector_binary_join_groupby.q,\

http://git-wip-us.apache.org/repos/asf/hive/blob/0ec5185a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index ad4efef..f05407d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -6678,10 +6678,6 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       LOG.debug("Couldn't find table " + tableName + " in insertIntoTable");
       throw new SemanticException(ErrorMsg.NO_INSERT_OVERWRITE_WITH_ACID.getMsg());
     }
-    if (conf.getBoolVar(ConfVars.HIVE_VECTORIZATION_ENABLED)) {
-      LOG.info("Turning off vectorization for acid write operation");
-      conf.setBoolVar(ConfVars.HIVE_VECTORIZATION_ENABLED, false);
-    }
     LOG.info("Modifying config values for ACID write");
     conf.setBoolVar(ConfVars.HIVEOPTREDUCEDEDUPLICATION, true);
     conf.setIntVar(ConfVars.HIVEOPTREDUCEDEDUPLICATIONMINREDUCER, 1);

http://git-wip-us.apache.org/repos/asf/hive/blob/0ec5185a/ql/src/test/queries/clientpositive/vector_acid3.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/vector_acid3.q b/ql/src/test/queries/clientpositive/vector_acid3.q
new file mode 100644
index 0000000..d4313f4
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/vector_acid3.q
@@ -0,0 +1,17 @@
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+set hive.exec.dynamic.partition.mode=nonstrict;
+set hive.exec.dynamic.partition=true;
+set hive.vectorized.execution.enabled=true;
+
+drop table if exists testacid1;
+
+create table testacid1(id int) clustered by (id) into 2 buckets stored as orc tblproperties("transactional"="true");
+
+insert into table testacid1 values (1),(2),(3),(4);
+
+set hive.compute.query.using.stats=false;
+
+set hive.vectorized.execution.enabled;
+
+select count(1) from testacid1;

http://git-wip-us.apache.org/repos/asf/hive/blob/0ec5185a/ql/src/test/results/clientpositive/tez/vector_acid3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vector_acid3.q.out b/ql/src/test/results/clientpositive/tez/vector_acid3.q.out
new file mode 100644
index 0000000..4299c73
--- /dev/null
+++ b/ql/src/test/results/clientpositive/tez/vector_acid3.q.out
@@ -0,0 +1,31 @@
+PREHOOK: query: drop table if exists testacid1
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists testacid1
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: create table testacid1(id int) clustered by (id) into 2 buckets stored as orc tblproperties("transactional"="true")
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@testacid1
+POSTHOOK: query: create table testacid1(id int) clustered by (id) into 2 buckets stored as orc tblproperties("transactional"="true")
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@testacid1
+PREHOOK: query: insert into table testacid1 values (1),(2),(3),(4)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__1
+PREHOOK: Output: default@testacid1
+POSTHOOK: query: insert into table testacid1 values (1),(2),(3),(4)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__1
+POSTHOOK: Output: default@testacid1
+POSTHOOK: Lineage: testacid1.id EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+hive.vectorized.execution.enabled=true
+PREHOOK: query: select count(1) from testacid1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@testacid1
+#### A masked pattern was here ####
+POSTHOOK: query: select count(1) from testacid1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@testacid1
+#### A masked pattern was here ####
+4

http://git-wip-us.apache.org/repos/asf/hive/blob/0ec5185a/ql/src/test/results/clientpositive/vector_acid3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_acid3.q.out b/ql/src/test/results/clientpositive/vector_acid3.q.out
new file mode 100644
index 0000000..4299c73
--- /dev/null
+++ b/ql/src/test/results/clientpositive/vector_acid3.q.out
@@ -0,0 +1,31 @@
+PREHOOK: query: drop table if exists testacid1
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists testacid1
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: create table testacid1(id int) clustered by (id) into 2 buckets stored as orc tblproperties("transactional"="true")
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@testacid1
+POSTHOOK: query: create table testacid1(id int) clustered by (id) into 2 buckets stored as orc tblproperties("transactional"="true")
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@testacid1
+PREHOOK: query: insert into table testacid1 values (1),(2),(3),(4)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__1
+PREHOOK: Output: default@testacid1
+POSTHOOK: query: insert into table testacid1 values (1),(2),(3),(4)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__1
+POSTHOOK: Output: default@testacid1
+POSTHOOK: Lineage: testacid1.id EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+hive.vectorized.execution.enabled=true
+PREHOOK: query: select count(1) from testacid1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@testacid1
+#### A masked pattern was here ####
+POSTHOOK: query: select count(1) from testacid1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@testacid1
+#### A masked pattern was here ####
+4


[03/43] hive git commit: HIVE-11172 : Vectorization wrong results for aggregate query with where clause without group by (Hari Subramaniyan, reviewed by Matt McCline)

Posted by xu...@apache.org.
HIVE-11172 : Vectorization wrong results for aggregate query with where clause without group by (Hari Subramaniyan, reviewed by Matt McCline)


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

Branch: refs/heads/spark
Commit: 55853b4160f545c5b9a327b802126be46a4c619f
Parents: 0944865
Author: Hari Subramaniyan <ha...@apache.org>
Authored: Tue Jul 21 03:03:45 2015 -0700
Committer: Hari Subramaniyan <ha...@apache.org>
Committed: Tue Jul 21 03:03:45 2015 -0700

----------------------------------------------------------------------
 .../UDAFTemplates/VectorUDAFMinMaxString.txt    |  3 +-
 .../vector_aggregate_without_gby.q              | 14 +++
 .../vector_aggregate_without_gby.q.out          | 96 ++++++++++++++++++++
 3 files changed, 112 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/55853b41/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxString.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxString.txt b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxString.txt
index 7e0dda6..cdce457 100644
--- a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxString.txt
+++ b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxString.txt
@@ -319,7 +319,8 @@ public class <ClassName> extends VectorAggregateExpression {
         int batchSize,
         int[] selected) {
 
-      for (int i=0; i< batchSize; ++i) {
+      for (int j=0; j< batchSize; ++j) {
+        int i = selected[j];
         myagg.checkValue(inputColumn.vector[i],
           inputColumn.start[i],
           inputColumn.length[i]);

http://git-wip-us.apache.org/repos/asf/hive/blob/55853b41/ql/src/test/queries/clientpositive/vector_aggregate_without_gby.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/vector_aggregate_without_gby.q b/ql/src/test/queries/clientpositive/vector_aggregate_without_gby.q
new file mode 100644
index 0000000..9a026ed
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/vector_aggregate_without_gby.q
@@ -0,0 +1,14 @@
+create table testvec(id int, dt int, greg_dt string) stored as orc;
+insert into table testvec
+values 
+(1,20150330, '2015-03-30'),
+(2,20150301, '2015-03-01'),
+(3,20150502, '2015-05-02'),
+(4,20150401, '2015-04-01'),
+(5,20150313, '2015-03-13'),
+(6,20150314, '2015-03-14'),
+(7,20150404, '2015-04-04');
+set hive.vectorized.execution.enabled=true;
+set hive.map.aggr=true;
+explain select max(dt), max(greg_dt) from testvec where id=5;
+select max(dt), max(greg_dt) from testvec where id=5;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/55853b41/ql/src/test/results/clientpositive/vector_aggregate_without_gby.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_aggregate_without_gby.q.out b/ql/src/test/results/clientpositive/vector_aggregate_without_gby.q.out
new file mode 100644
index 0000000..1175cb8
--- /dev/null
+++ b/ql/src/test/results/clientpositive/vector_aggregate_without_gby.q.out
@@ -0,0 +1,96 @@
+PREHOOK: query: create table testvec(id int, dt int, greg_dt string) stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@testvec
+POSTHOOK: query: create table testvec(id int, dt int, greg_dt string) stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@testvec
+PREHOOK: query: insert into table testvec
+values 
+(1,20150330, '2015-03-30'),
+(2,20150301, '2015-03-01'),
+(3,20150502, '2015-05-02'),
+(4,20150401, '2015-04-01'),
+(5,20150313, '2015-03-13'),
+(6,20150314, '2015-03-14'),
+(7,20150404, '2015-04-04')
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__1
+PREHOOK: Output: default@testvec
+POSTHOOK: query: insert into table testvec
+values 
+(1,20150330, '2015-03-30'),
+(2,20150301, '2015-03-01'),
+(3,20150502, '2015-05-02'),
+(4,20150401, '2015-04-01'),
+(5,20150313, '2015-03-13'),
+(6,20150314, '2015-03-14'),
+(7,20150404, '2015-04-04')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__1
+POSTHOOK: Output: default@testvec
+POSTHOOK: Lineage: testvec.dt EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+POSTHOOK: Lineage: testvec.greg_dt SIMPLE [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
+POSTHOOK: Lineage: testvec.id EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+PREHOOK: query: explain select max(dt), max(greg_dt) from testvec where id=5
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select max(dt), max(greg_dt) from testvec where id=5
+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: testvec
+            Statistics: Num rows: 7 Data size: 714 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: (id = 5) (type: boolean)
+              Statistics: Num rows: 3 Data size: 306 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: dt (type: int), greg_dt (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 3 Data size: 306 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: max(_col0), max(_col1)
+                  mode: hash
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col0 (type: int), _col1 (type: string)
+      Execution mode: vectorized
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: max(VALUE._col0), max(VALUE._col1)
+          mode: mergepartial
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: NONE
+            table:
+                input format: org.apache.hadoop.mapred.TextInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select max(dt), max(greg_dt) from testvec where id=5
+PREHOOK: type: QUERY
+PREHOOK: Input: default@testvec
+#### A masked pattern was here ####
+POSTHOOK: query: select max(dt), max(greg_dt) from testvec where id=5
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@testvec
+#### A masked pattern was here ####
+20150313	2015-03-13


[13/43] hive git commit: HIVE-11310: Avoid expensive AST tree conversion to String for expressions in WHERE clause (Jesus Camacho Rodriguez, reviewed by Hari Sankar Sivarama Subramaniyan)

Posted by xu...@apache.org.
HIVE-11310: Avoid expensive AST tree conversion to String for expressions in WHERE clause (Jesus Camacho Rodriguez, reviewed by Hari Sankar Sivarama Subramaniyan)


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

Branch: refs/heads/spark
Commit: 57242e3430decfc9984230e3bf9d26f0304855d6
Parents: 83be12f
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Tue Jul 21 10:33:22 2015 +0100
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Wed Jul 22 16:24:52 2015 +0100

----------------------------------------------------------------------
 .../calcite/translator/JoinTypeCheckCtx.java    |  2 +-
 .../apache/hadoop/hive/ql/parse/ParseUtils.java | 51 +++++++++++++++++++-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  | 32 +++++++-----
 .../hadoop/hive/ql/parse/TypeCheckCtx.java      | 15 +++++-
 .../hive/ql/parse/TypeCheckProcFactory.java     |  4 ++
 5 files changed, 87 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/57242e34/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/JoinTypeCheckCtx.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/JoinTypeCheckCtx.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/JoinTypeCheckCtx.java
index bbd4723..dccd1d9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/JoinTypeCheckCtx.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/JoinTypeCheckCtx.java
@@ -53,7 +53,7 @@ public class JoinTypeCheckCtx extends TypeCheckCtx {
 
   public JoinTypeCheckCtx(RowResolver leftRR, RowResolver rightRR, JoinType hiveJoinType)
       throws SemanticException {
-    super(RowResolver.getCombinedRR(leftRR, rightRR), false, false, false, false, false, false,
+    super(RowResolver.getCombinedRR(leftRR, rightRR), true, false, false, false, false, false, false,
         false, false);
     this.inputRRLst = ImmutableList.of(leftRR, rightRR);
     this.outerJoin = (hiveJoinType == JoinType.LEFTOUTER) || (hiveJoinType == JoinType.RIGHTOUTER)

http://git-wip-us.apache.org/repos/asf/hive/blob/57242e34/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java
index 18f0180..bf1b5d4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseUtils.java
@@ -18,9 +18,16 @@
 
 package org.apache.hadoop.hive.ql.parse;
 
-import java.util.*;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Queue;
+import java.util.Set;
+import java.util.Stack;
 
-import org.apache.hadoop.hive.common.JavaUtils;
+import org.antlr.runtime.tree.Tree;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.ql.ErrorMsg;
@@ -266,4 +273,44 @@ public final class ParseUtils {
 
       return false;
     }
+
+    public static boolean sameTree(ASTNode node, ASTNode otherNode) {
+      if (node == null && otherNode == null) {
+        return true;
+      }
+      if ((node == null && otherNode != null) ||
+              (node != null && otherNode == null)) {
+        return false;
+      }
+
+      Stack<Tree> stack = new Stack<Tree>();
+      stack.push(node);
+      Stack<Tree> otherStack = new Stack<Tree>();
+      otherStack.push(otherNode);
+
+      while (!stack.empty() && !otherStack.empty()) {
+        Tree p = stack.pop();
+        Tree otherP = otherStack.pop();
+
+        if (p.isNil() != otherP.isNil()) {
+          return false;
+        }
+        if (!p.isNil()) {
+          if (!p.toString().equals(otherP.toString())) {
+            return false;
+          }
+        }
+        if (p.getChildCount() != otherP.getChildCount()) {
+          return false;
+        }
+        for (int i = p.getChildCount()-1; i >= 0; i--) {
+          Tree t = p.getChild(i);
+          stack.push(t);
+          Tree otherT = otherP.getChild(i);
+          otherStack.push(otherT);
+        }
+      }
+
+      return stack.empty() && otherStack.empty();
+    }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/57242e34/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index aab4250..0c191da 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -2625,7 +2625,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
      * so we invoke genFilterPlan to handle SubQuery algebraic transformation,
      * just as is done for SubQuery predicates appearing in the Where Clause.
      */
-    Operator output = genFilterPlan(condn, qb, input, aliasToOpInfo, true);
+    Operator output = genFilterPlan(condn, qb, input, aliasToOpInfo, true, false);
     output = putOpInsertMap(output, inputRR);
     return output;
   }
@@ -2644,7 +2644,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
   @SuppressWarnings("nls")
   private Operator genFilterPlan(ASTNode searchCond, QB qb, Operator input,
       Map<String, Operator> aliasToOpInfo,
-      boolean forHavingClause)
+      boolean forHavingClause, boolean forGroupByClause)
       throws SemanticException {
 
     OpParseContext inputCtx = opParseCtx.get(input);
@@ -2786,7 +2786,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       }
     }
 
-    return genFilterPlan(qb, searchCond, input);
+    return genFilterPlan(qb, searchCond, input, forHavingClause || forGroupByClause);
   }
 
   /**
@@ -2800,13 +2800,13 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
    *          the input operator
    */
   @SuppressWarnings("nls")
-  private Operator genFilterPlan(QB qb, ASTNode condn, Operator input)
+  private Operator genFilterPlan(QB qb, ASTNode condn, Operator input, boolean useCaching)
       throws SemanticException {
 
     OpParseContext inputCtx = opParseCtx.get(input);
     RowResolver inputRR = inputCtx.getRowResolver();
     Operator output = putOpInsertMap(OperatorFactory.getAndMakeChild(
-        new FilterDesc(genExprNodeDesc(condn, inputRR), false), new RowSchema(
+        new FilterDesc(genExprNodeDesc(condn, inputRR, useCaching), false), new RowSchema(
             inputRR.getColumnInfos()), input), inputRR);
 
     if (LOG.isDebugEnabled()) {
@@ -5414,7 +5414,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
 
       if (parseInfo.getWhrForClause(dest) != null) {
         ASTNode whereExpr = qb.getParseInfo().getWhrForClause(dest);
-        curr = genFilterPlan((ASTNode) whereExpr.getChild(0), qb, forwardOp, aliasToOpInfo, false);
+        curr = genFilterPlan((ASTNode) whereExpr.getChild(0), qb, forwardOp, aliasToOpInfo, false, true);
       }
 
       // Generate GroupbyOperator
@@ -7559,7 +7559,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     if ( joinSrcOp != null ) {
       ArrayList<ASTNode> filter = joinTree.getFiltersForPushing().get(0);
       for (ASTNode cond : filter) {
-        joinSrcOp = genFilterPlan(qb, cond, joinSrcOp);
+        joinSrcOp = genFilterPlan(qb, cond, joinSrcOp, false);
       }
     }
 
@@ -7615,7 +7615,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
 
     Operator op = joinOp;
     for(ASTNode condn : joinTree.getPostJoinFilters() ) {
-      op = genFilterPlan(qb, condn, op);
+      op = genFilterPlan(qb, condn, op, false);
     }
     return op;
   }
@@ -7788,7 +7788,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         Operator srcOp = map.get(src);
         ArrayList<ASTNode> filter = filters.get(pos);
         for (ASTNode cond : filter) {
-          srcOp = genFilterPlan(qb, cond, srcOp);
+          srcOp = genFilterPlan(qb, cond, srcOp, false);
         }
         map.put(src, srcOp);
       }
@@ -8831,7 +8831,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
 
             if (qbp.getWhrForClause(dest) != null) {
               ASTNode whereExpr = qb.getParseInfo().getWhrForClause(dest);
-              curr = genFilterPlan((ASTNode) whereExpr.getChild(0), qb, curr, aliasToOpInfo, false);
+              curr = genFilterPlan((ASTNode) whereExpr.getChild(0), qb, curr, aliasToOpInfo, false, false);
             }
             // Preserve operator before the GBY - we'll use it to resolve '*'
             Operator<?> gbySource = curr;
@@ -10425,7 +10425,12 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       throws SemanticException {
     // Since the user didn't supply a customized type-checking context,
     // use default settings.
-    TypeCheckCtx tcCtx = new TypeCheckCtx(input);
+    return genExprNodeDesc(expr, input, true);
+  }
+
+  public ExprNodeDesc genExprNodeDesc(ASTNode expr, RowResolver input, boolean useCaching)
+      throws SemanticException {
+    TypeCheckCtx tcCtx = new TypeCheckCtx(input, useCaching);
     return genExprNodeDesc(expr, input, tcCtx);
   }
 
@@ -10453,7 +10458,10 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     // build the exprNodeFuncDesc with recursively built children.
 
     // If the current subExpression is pre-calculated, as in Group-By etc.
-    ExprNodeDesc cached = getExprNodeDescCached(expr, input);
+    ExprNodeDesc cached = null;
+    if (tcCtx.isUseCaching()) {
+      cached = getExprNodeDescCached(expr, input);
+    }
     if (cached == null) {
       Map<ASTNode, ExprNodeDesc> allExprs = genAllExprNodeDesc(expr, input, tcCtx);
       return allExprs.get(expr);

http://git-wip-us.apache.org/repos/asf/hive/blob/57242e34/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckCtx.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckCtx.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckCtx.java
index b19e2bf..8ad28be 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckCtx.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckCtx.java
@@ -35,6 +35,8 @@ public class TypeCheckCtx implements NodeProcessorCtx {
    */
   private RowResolver inputRR;
 
+  private final boolean useCaching;
+
   /**
    * Receives translations which will need to be applied during unparse.
    */
@@ -77,15 +79,20 @@ public class TypeCheckCtx implements NodeProcessorCtx {
    *          The input row resolver of the previous operator.
    */
   public TypeCheckCtx(RowResolver inputRR) {
-    this(inputRR, false, true, true, true, true, true, true, true);
+    this(inputRR, true);
+  }
+
+  public TypeCheckCtx(RowResolver inputRR, boolean useCaching) {
+    this(inputRR, useCaching, false, true, true, true, true, true, true, true);
   }
 
-  public TypeCheckCtx(RowResolver inputRR, boolean allowStatefulFunctions,
+  public TypeCheckCtx(RowResolver inputRR, boolean useCaching, boolean allowStatefulFunctions,
       boolean allowDistinctFunctions, boolean allowGBExprElimination, boolean allowAllColRef,
       boolean allowFunctionStar, boolean allowWindowing,
       boolean allowIndexExpr, boolean allowSubQueryExpr) {
     setInputRR(inputRR);
     error = null;
+    this.useCaching = useCaching;
     this.allowStatefulFunctions = allowStatefulFunctions;
     this.allowDistinctFunctions = allowDistinctFunctions;
     this.allowGBExprElimination = allowGBExprElimination;
@@ -198,4 +205,8 @@ public class TypeCheckCtx implements NodeProcessorCtx {
   public boolean getallowSubQueryExpr() {
     return allowSubQueryExpr;
   }
+
+  public boolean isUseCaching() {
+    return useCaching;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/57242e34/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
index 0e97530..d823f03 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
@@ -124,6 +124,10 @@ public class TypeCheckProcFactory {
     ASTNode expr = (ASTNode) nd;
     TypeCheckCtx ctx = (TypeCheckCtx) procCtx;
 
+    if (!ctx.isUseCaching()) {
+      return null;
+    }
+
     RowResolver input = ctx.getInputRR();
     ExprNodeDesc desc = null;
 


[07/43] hive git commit: HIVE-11303: Getting Tez LimitExceededException after dag execution on large query (Jason Dere, reviewed by Vikram Dixit)

Posted by xu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/04d54f61/ql/src/test/results/clientpositive/tez/tez_vector_dynpart_hashjoin_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/tez_vector_dynpart_hashjoin_1.q.out b/ql/src/test/results/clientpositive/tez/tez_vector_dynpart_hashjoin_1.q.out
new file mode 100644
index 0000000..ba35e4c
--- /dev/null
+++ b/ql/src/test/results/clientpositive/tez/tez_vector_dynpart_hashjoin_1.q.out
@@ -0,0 +1,804 @@
+PREHOOK: query: -- First try with regular mergejoin
+explain
+select
+  *
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+order by a.cint
+PREHOOK: type: QUERY
+POSTHOOK: query: -- First try with regular mergejoin
+explain
+select
+  *
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+order by a.cint
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+      Edges:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (cint BETWEEN 1000000 AND 3000000 and cint is not null) (type: boolean)
+                    Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cfloat (type: float), cdouble (type: double), cstring1 (type: string), cstring2 (type: string), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), cboolean1 (type: boolean), cboolean2 (type: boolean)
+                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
+                      Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col2 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col2 (type: int)
+                        Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: ((cbigint is not null and cint BETWEEN 1000000 AND 3000000) and cint is not null) (type: boolean)
+                    Statistics: Num rows: 1536 Data size: 330245 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cfloat (type: float), cdouble (type: double), cstring1 (type: string), cstring2 (type: string), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), cboolean1 (type: boolean), cboolean2 (type: boolean)
+                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
+                      Statistics: Num rows: 1536 Data size: 330245 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col2 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col2 (type: int)
+                        Statistics: Num rows: 1536 Data size: 330245 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
+        Reducer 2 
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col2 (type: int)
+                  1 _col2 (type: int)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23
+                Statistics: Num rows: 3379 Data size: 726540 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col2 (type: int)
+                  sort order: +
+                  Statistics: Num rows: 3379 Data size: 726540 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean), _col12 (type: tinyint), _col13 (type: smallint), _col14 (type: int), _col15 (type: bigint), _col16 (type: float), _col17 (type: double), _col18 (type: string), _col19 (type: string), _col20 (type: timestamp), _col21 (type: timestamp), _col22 (type: boolean), _col23 (type: boolean)
+        Reducer 3 
+            Reduce Operator Tree:
+              Select Operator
+                expressions: VALUE._col0 (type: tinyint), VALUE._col1 (type: smallint), KEY.reducesinkkey0 (type: int), VALUE._col2 (type: bigint), VALUE._col3 (type: float), VALUE._col4 (type: double), VALUE._col5 (type: string), VALUE._col6 (type: string), VALUE._col7 (type: timestamp), VALUE._col8 (type: timestamp), VALUE._col9 (type: boolean), VALUE._col10 (type: boolean), VALUE._col11 (type: tinyint), VALUE._col12 (type: smallint), VALUE._col13 (type: int), VALUE._col14 (type: bigint), VALUE._col15 (type: float), VALUE._col16 (type: double), VALUE._col17 (type: string), VALUE._col18 (type: string), VALUE._col19 (type: timestamp), VALUE._col20 (type: timestamp), VALUE._col21 (type: boolean), VALUE._col22 (type: boolean)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23
+                Statistics: Num rows: 3379 Data size: 726540 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 3379 Data size: 726540 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select
+  *
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+order by a.cint
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+POSTHOOK: query: select
+  *
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+order by a.cint
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+11	NULL	1000828	1531084669	11.0	NULL	wM316f6NqGIkoP388j3F6	poWQQo3Upvt3Wh	1969-12-31 16:00:02.351	NULL	false	true	11	NULL	1000828	1531084669	11.0	NULL	wM316f6NqGIkoP388j3F6	poWQQo3Upvt3Wh	1969-12-31 16:00:02.351	NULL	false	true
+NULL	-3799	1248059	1864027286	NULL	-3799.0	Uhps6mMh3IfHB3j7yH62K	4KWs6gw7lv2WYd66P	NULL	1969-12-31 15:59:54.622	false	true	NULL	-3799	1248059	1864027286	NULL	-3799.0	Uhps6mMh3IfHB3j7yH62K	4KWs6gw7lv2WYd66P	NULL	1969-12-31 15:59:54.622	false	true
+NULL	10782	1286921	1864027286	NULL	10782.0	ODLrXI8882q8LS8	4KWs6gw7lv2WYd66P	NULL	1969-12-31 15:59:52.138	true	true	NULL	10782	1286921	1864027286	NULL	10782.0	ODLrXI8882q8LS8	4KWs6gw7lv2WYd66P	NULL	1969-12-31 15:59:52.138	true	true
+NULL	-13036	1288927	-1645852809	NULL	-13036.0	yinBY725P7V2	xH7445Rals48VOulSyR5F	NULL	1969-12-31 16:00:00.763	true	false	NULL	-13036	1288927	-1645852809	NULL	-13036.0	yinBY725P7V2	xH7445Rals48VOulSyR5F	NULL	1969-12-31 16:00:00.763	true	false
+11	NULL	1310786	-413875656	11.0	NULL	W0rvA4H1xn0xMG4uk0	8yVVjG	1969-12-31 16:00:02.351	NULL	false	true	11	NULL	1310786	-413875656	11.0	NULL	W0rvA4H1xn0xMG4uk0	8yVVjG	1969-12-31 16:00:02.351	NULL	false	true
+-51	NULL	2089466	-240556350	-51.0	NULL	cXX24dH7tblSj46j2g	C31eea0wrHHqvj	1969-12-31 16:00:08.451	NULL	true	true	-51	NULL	2089466	-240556350	-51.0	NULL	cXX24dH7tblSj46j2g	C31eea0wrHHqvj	1969-12-31 16:00:08.451	NULL	true	true
+NULL	-8915	2101183	1864027286	NULL	-8915.0	x7By66525	4KWs6gw7lv2WYd66P	NULL	1969-12-31 16:00:05.831	false	true	NULL	-8915	2101183	1864027286	NULL	-8915.0	x7By66525	4KWs6gw7lv2WYd66P	NULL	1969-12-31 16:00:05.831	false	true
+8	NULL	2229621	-381406148	8.0	NULL	q7onkS7QRPh5ghOK	oKb0bi	1969-12-31 16:00:15.892	NULL	true	false	8	NULL	2229621	-381406148	8.0	NULL	q7onkS7QRPh5ghOK	oKb0bi	1969-12-31 16:00:15.892	NULL	true	false
+8	NULL	2433892	-1611863517	8.0	NULL	674ILv3V2TxFqXP6wSbL	VLprkK2XfX	1969-12-31 16:00:15.892	NULL	false	true	8	NULL	2433892	-1611863517	8.0	NULL	674ILv3V2TxFqXP6wSbL	VLprkK2XfX	1969-12-31 16:00:15.892	NULL	false	true
+-51	NULL	2949963	-1580871111	-51.0	NULL	0K68k3bdl7jO7	TPPAu	1969-12-31 16:00:08.451	NULL	true	false	-51	NULL	2949963	-1580871111	-51.0	NULL	0K68k3bdl7jO7	TPPAu	1969-12-31 16:00:08.451	NULL	true	false
+PREHOOK: query: explain
+select
+  count(*)
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select
+  count(*)
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+      Edges:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (cint BETWEEN 1000000 AND 3000000 and cint is not null) (type: boolean)
+                    Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: cint (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 3072 Data size: 660491 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: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: ((cbigint is not null and cint BETWEEN 1000000 AND 3000000) and cint is not null) (type: boolean)
+                    Statistics: Num rows: 1536 Data size: 330245 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: cint (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1536 Data size: 330245 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: 1536 Data size: 330245 Basic stats: COMPLETE Column stats: NONE
+        Reducer 2 
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: int)
+                  1 _col0 (type: int)
+                Statistics: Num rows: 3379 Data size: 726540 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: count()
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col0 (type: bigint)
+        Reducer 3 
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select
+  count(*)
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+POSTHOOK: query: select
+  count(*)
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+10
+PREHOOK: query: explain
+select
+  a.csmallint, count(*) c1
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+group by a.csmallint
+order by c1
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select
+  a.csmallint, count(*) c1
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+group by a.csmallint
+order by c1
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+      Edges:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+        Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (cint BETWEEN 1000000 AND 3000000 and cint is not null) (type: boolean)
+                    Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: csmallint (type: smallint), cint (type: int)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col1 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col1 (type: int)
+                        Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: smallint)
+        Map 5 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: ((cbigint is not null and cint BETWEEN 1000000 AND 3000000) and cint is not null) (type: boolean)
+                    Statistics: Num rows: 1536 Data size: 330245 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: cint (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1536 Data size: 330245 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: 1536 Data size: 330245 Basic stats: COMPLETE Column stats: NONE
+        Reducer 2 
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col1 (type: int)
+                  1 _col0 (type: int)
+                outputColumnNames: _col0
+                Statistics: Num rows: 3379 Data size: 726540 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: count()
+                  keys: _col0 (type: smallint)
+                  mode: hash
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 3379 Data size: 726540 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: smallint)
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: smallint)
+                    Statistics: Num rows: 3379 Data size: 726540 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col1 (type: bigint)
+        Reducer 3 
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                keys: KEY._col0 (type: smallint)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1689 Data size: 363162 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col1 (type: bigint)
+                  sort order: +
+                  Statistics: Num rows: 1689 Data size: 363162 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col0 (type: smallint)
+        Reducer 4 
+            Reduce Operator Tree:
+              Select Operator
+                expressions: VALUE._col0 (type: smallint), KEY.reducesinkkey0 (type: bigint)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1689 Data size: 363162 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1689 Data size: 363162 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select
+  a.csmallint, count(*) c1
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+group by a.csmallint
+order by c1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+POSTHOOK: query: select
+  a.csmallint, count(*) c1
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+group by a.csmallint
+order by c1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+10782	1
+-3799	1
+-8915	1
+-13036	1
+NULL	6
+PREHOOK: query: -- Try with dynamically partitioned hashjoin
+explain
+select
+  *
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+order by a.cint
+PREHOOK: type: QUERY
+POSTHOOK: query: -- Try with dynamically partitioned hashjoin
+explain
+select
+  *
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+order by a.cint
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+      Edges:
+        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Map 4 (CUSTOM_SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (cint BETWEEN 1000000 AND 3000000 and cint is not null) (type: boolean)
+                    Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cfloat (type: float), cdouble (type: double), cstring1 (type: string), cstring2 (type: string), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), cboolean1 (type: boolean), cboolean2 (type: boolean)
+                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
+                      Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col2 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col2 (type: int)
+                        Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
+            Execution mode: vectorized
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: ((cbigint is not null and cint BETWEEN 1000000 AND 3000000) and cint is not null) (type: boolean)
+                    Statistics: Num rows: 1536 Data size: 330245 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cfloat (type: float), cdouble (type: double), cstring1 (type: string), cstring2 (type: string), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), cboolean1 (type: boolean), cboolean2 (type: boolean)
+                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
+                      Statistics: Num rows: 1536 Data size: 330245 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col2 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col2 (type: int)
+                        Statistics: Num rows: 1536 Data size: 330245 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
+            Execution mode: vectorized
+        Reducer 2 
+            Reduce Operator Tree:
+              Map Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 KEY.reducesinkkey0 (type: int)
+                  1 KEY.reducesinkkey0 (type: int)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23
+                input vertices:
+                  1 Map 4
+                Statistics: Num rows: 3379 Data size: 726540 Basic stats: COMPLETE Column stats: NONE
+                HybridGraceHashJoin: true
+                Reduce Output Operator
+                  key expressions: _col2 (type: int)
+                  sort order: +
+                  Statistics: Num rows: 3379 Data size: 726540 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean), _col12 (type: tinyint), _col13 (type: smallint), _col14 (type: int), _col15 (type: bigint), _col16 (type: float), _col17 (type: double), _col18 (type: string), _col19 (type: string), _col20 (type: timestamp), _col21 (type: timestamp), _col22 (type: boolean), _col23 (type: boolean)
+            Execution mode: vectorized
+        Reducer 3 
+            Reduce Operator Tree:
+              Select Operator
+                expressions: VALUE._col0 (type: tinyint), VALUE._col1 (type: smallint), KEY.reducesinkkey0 (type: int), VALUE._col2 (type: bigint), VALUE._col3 (type: float), VALUE._col4 (type: double), VALUE._col5 (type: string), VALUE._col6 (type: string), VALUE._col7 (type: timestamp), VALUE._col8 (type: timestamp), VALUE._col9 (type: boolean), VALUE._col10 (type: boolean), VALUE._col11 (type: tinyint), VALUE._col12 (type: smallint), VALUE._col13 (type: int), VALUE._col14 (type: bigint), VALUE._col15 (type: float), VALUE._col16 (type: double), VALUE._col17 (type: string), VALUE._col18 (type: string), VALUE._col19 (type: timestamp), VALUE._col20 (type: timestamp), VALUE._col21 (type: boolean), VALUE._col22 (type: boolean)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23
+                Statistics: Num rows: 3379 Data size: 726540 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 3379 Data size: 726540 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            Execution mode: vectorized
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select
+  *
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+order by a.cint
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+POSTHOOK: query: select
+  *
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+order by a.cint
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+11	NULL	1000828	1531084669	11.0	NULL	wM316f6NqGIkoP388j3F6	poWQQo3Upvt3Wh	1969-12-31 16:00:02.351	NULL	false	true	11	NULL	1000828	1531084669	11.0	NULL	wM316f6NqGIkoP388j3F6	poWQQo3Upvt3Wh	1969-12-31 16:00:02.351	NULL	false	true
+NULL	-3799	1248059	1864027286	NULL	-3799.0	Uhps6mMh3IfHB3j7yH62K	4KWs6gw7lv2WYd66P	NULL	1969-12-31 15:59:54.622	false	true	NULL	-3799	1248059	1864027286	NULL	-3799.0	Uhps6mMh3IfHB3j7yH62K	4KWs6gw7lv2WYd66P	NULL	1969-12-31 15:59:54.622	false	true
+NULL	10782	1286921	1864027286	NULL	10782.0	ODLrXI8882q8LS8	4KWs6gw7lv2WYd66P	NULL	1969-12-31 15:59:52.138	true	true	NULL	10782	1286921	1864027286	NULL	10782.0	ODLrXI8882q8LS8	4KWs6gw7lv2WYd66P	NULL	1969-12-31 15:59:52.138	true	true
+NULL	-13036	1288927	-1645852809	NULL	-13036.0	yinBY725P7V2	xH7445Rals48VOulSyR5F	NULL	1969-12-31 16:00:00.763	true	false	NULL	-13036	1288927	-1645852809	NULL	-13036.0	yinBY725P7V2	xH7445Rals48VOulSyR5F	NULL	1969-12-31 16:00:00.763	true	false
+11	NULL	1310786	-413875656	11.0	NULL	W0rvA4H1xn0xMG4uk0	8yVVjG	1969-12-31 16:00:02.351	NULL	false	true	11	NULL	1310786	-413875656	11.0	NULL	W0rvA4H1xn0xMG4uk0	8yVVjG	1969-12-31 16:00:02.351	NULL	false	true
+-51	NULL	2089466	-240556350	-51.0	NULL	cXX24dH7tblSj46j2g	C31eea0wrHHqvj	1969-12-31 16:00:08.451	NULL	true	true	-51	NULL	2089466	-240556350	-51.0	NULL	cXX24dH7tblSj46j2g	C31eea0wrHHqvj	1969-12-31 16:00:08.451	NULL	true	true
+NULL	-8915	2101183	1864027286	NULL	-8915.0	x7By66525	4KWs6gw7lv2WYd66P	NULL	1969-12-31 16:00:05.831	false	true	NULL	-8915	2101183	1864027286	NULL	-8915.0	x7By66525	4KWs6gw7lv2WYd66P	NULL	1969-12-31 16:00:05.831	false	true
+8	NULL	2229621	-381406148	8.0	NULL	q7onkS7QRPh5ghOK	oKb0bi	1969-12-31 16:00:15.892	NULL	true	false	8	NULL	2229621	-381406148	8.0	NULL	q7onkS7QRPh5ghOK	oKb0bi	1969-12-31 16:00:15.892	NULL	true	false
+8	NULL	2433892	-1611863517	8.0	NULL	674ILv3V2TxFqXP6wSbL	VLprkK2XfX	1969-12-31 16:00:15.892	NULL	false	true	8	NULL	2433892	-1611863517	8.0	NULL	674ILv3V2TxFqXP6wSbL	VLprkK2XfX	1969-12-31 16:00:15.892	NULL	false	true
+-51	NULL	2949963	-1580871111	-51.0	NULL	0K68k3bdl7jO7	TPPAu	1969-12-31 16:00:08.451	NULL	true	false	-51	NULL	2949963	-1580871111	-51.0	NULL	0K68k3bdl7jO7	TPPAu	1969-12-31 16:00:08.451	NULL	true	false
+PREHOOK: query: explain
+select
+  count(*)
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select
+  count(*)
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+      Edges:
+        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Map 4 (CUSTOM_SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (cint BETWEEN 1000000 AND 3000000 and cint is not null) (type: boolean)
+                    Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: cint (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 3072 Data size: 660491 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: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE
+            Execution mode: vectorized
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: ((cbigint is not null and cint BETWEEN 1000000 AND 3000000) and cint is not null) (type: boolean)
+                    Statistics: Num rows: 1536 Data size: 330245 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: cint (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1536 Data size: 330245 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: 1536 Data size: 330245 Basic stats: COMPLETE Column stats: NONE
+            Execution mode: vectorized
+        Reducer 2 
+            Reduce Operator Tree:
+              Map Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 KEY.reducesinkkey0 (type: int)
+                  1 KEY.reducesinkkey0 (type: int)
+                input vertices:
+                  1 Map 4
+                Statistics: Num rows: 3379 Data size: 726540 Basic stats: COMPLETE Column stats: NONE
+                HybridGraceHashJoin: true
+                Group By Operator
+                  aggregations: count()
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col0 (type: bigint)
+            Execution mode: vectorized
+        Reducer 3 
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            Execution mode: vectorized
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select
+  count(*)
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+POSTHOOK: query: select
+  count(*)
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+10
+PREHOOK: query: explain
+select
+  a.csmallint, count(*) c1
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+group by a.csmallint
+order by c1
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select
+  a.csmallint, count(*) c1
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+group by a.csmallint
+order by c1
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+      Edges:
+        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Map 5 (CUSTOM_SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+        Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (cint BETWEEN 1000000 AND 3000000 and cint is not null) (type: boolean)
+                    Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: csmallint (type: smallint), cint (type: int)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col1 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col1 (type: int)
+                        Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: smallint)
+            Execution mode: vectorized
+        Map 5 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: ((cbigint is not null and cint BETWEEN 1000000 AND 3000000) and cint is not null) (type: boolean)
+                    Statistics: Num rows: 1536 Data size: 330245 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: cint (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1536 Data size: 330245 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: 1536 Data size: 330245 Basic stats: COMPLETE Column stats: NONE
+            Execution mode: vectorized
+        Reducer 2 
+            Reduce Operator Tree:
+              Map Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 KEY.reducesinkkey0 (type: int)
+                  1 KEY.reducesinkkey0 (type: int)
+                outputColumnNames: _col0
+                input vertices:
+                  1 Map 5
+                Statistics: Num rows: 3379 Data size: 726540 Basic stats: COMPLETE Column stats: NONE
+                HybridGraceHashJoin: true
+                Group By Operator
+                  aggregations: count()
+                  keys: _col0 (type: smallint)
+                  mode: hash
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 3379 Data size: 726540 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: smallint)
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: smallint)
+                    Statistics: Num rows: 3379 Data size: 726540 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col1 (type: bigint)
+            Execution mode: vectorized
+        Reducer 3 
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                keys: KEY._col0 (type: smallint)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1689 Data size: 363162 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col1 (type: bigint)
+                  sort order: +
+                  Statistics: Num rows: 1689 Data size: 363162 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col0 (type: smallint)
+            Execution mode: vectorized
+        Reducer 4 
+            Reduce Operator Tree:
+              Select Operator
+                expressions: VALUE._col0 (type: smallint), KEY.reducesinkkey0 (type: bigint)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1689 Data size: 363162 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1689 Data size: 363162 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            Execution mode: vectorized
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select
+  a.csmallint, count(*) c1
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+group by a.csmallint
+order by c1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+POSTHOOK: query: select
+  a.csmallint, count(*) c1
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+group by a.csmallint
+order by c1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+10782	1
+-3799	1
+-8915	1
+-13036	1
+NULL	6

http://git-wip-us.apache.org/repos/asf/hive/blob/04d54f61/ql/src/test/results/clientpositive/tez/tez_vector_dynpart_hashjoin_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/tez_vector_dynpart_hashjoin_2.q.out b/ql/src/test/results/clientpositive/tez/tez_vector_dynpart_hashjoin_2.q.out
new file mode 100644
index 0000000..e814103
--- /dev/null
+++ b/ql/src/test/results/clientpositive/tez/tez_vector_dynpart_hashjoin_2.q.out
@@ -0,0 +1,570 @@
+PREHOOK: query: -- Multiple tables, and change the order of the big table (alltypesorc)
+-- First try with regular mergejoin
+explain
+select
+  a.*
+from
+  alltypesorc a,
+  src b,
+  src c
+where
+  a.csmallint = cast(b.key as int) and a.csmallint = (cast(c.key as int) + 0)
+  and (a.csmallint < 100)
+order by a.csmallint, a.ctinyint, a.cint
+PREHOOK: type: QUERY
+POSTHOOK: query: -- Multiple tables, and change the order of the big table (alltypesorc)
+-- First try with regular mergejoin
+explain
+select
+  a.*
+from
+  alltypesorc a,
+  src b,
+  src c
+where
+  a.csmallint = cast(b.key as int) and a.csmallint = (cast(c.key as int) + 0)
+  and (a.csmallint < 100)
+order by a.csmallint, a.ctinyint, a.cint
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+      Edges:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: ((csmallint < 100) and UDFToInteger(csmallint) is not null) (type: boolean)
+                    Statistics: Num rows: 2048 Data size: 440327 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cfloat (type: float), cdouble (type: double), cstring1 (type: string), cstring2 (type: string), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), cboolean1 (type: boolean), cboolean2 (type: boolean)
+                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
+                      Statistics: Num rows: 2048 Data size: 440327 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: UDFToInteger(_col1) (type: int)
+                        sort order: +
+                        Map-reduce partition columns: UDFToInteger(_col1) (type: int)
+                        Statistics: Num rows: 2048 Data size: 440327 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: UDFToInteger(key) is not null (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
+                      Reduce Output Operator
+                        key expressions: UDFToInteger(_col0) (type: int)
+                        sort order: +
+                        Map-reduce partition columns: UDFToInteger(_col0) (type: int)
+                        Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+        Map 5 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (UDFToInteger(key) + 0) is not null (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
+                      Reduce Output Operator
+                        key expressions: (UDFToInteger(_col0) + 0) (type: int)
+                        sort order: +
+                        Map-reduce partition columns: (UDFToInteger(_col0) + 0) (type: int)
+                        Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+        Reducer 2 
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                     Inner Join 0 to 2
+                keys:
+                  0 UDFToInteger(_col1) (type: int)
+                  1 UDFToInteger(_col0) (type: int)
+                  2 (UDFToInteger(_col0) + 0) (type: int)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
+                Statistics: Num rows: 4505 Data size: 968719 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col1 (type: smallint), _col0 (type: tinyint), _col2 (type: int)
+                  sort order: +++
+                  Statistics: Num rows: 4505 Data size: 968719 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
+        Reducer 3 
+            Reduce Operator Tree:
+              Select Operator
+                expressions: KEY.reducesinkkey1 (type: tinyint), KEY.reducesinkkey0 (type: smallint), KEY.reducesinkkey2 (type: int), VALUE._col0 (type: bigint), VALUE._col1 (type: float), VALUE._col2 (type: double), VALUE._col3 (type: string), VALUE._col4 (type: string), VALUE._col5 (type: timestamp), VALUE._col6 (type: timestamp), VALUE._col7 (type: boolean), VALUE._col8 (type: boolean)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
+                Statistics: Num rows: 4505 Data size: 968719 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 4505 Data size: 968719 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select
+  a.*
+from
+  alltypesorc a,
+  src b,
+  src c
+where
+  a.csmallint = cast(b.key as int) and a.csmallint = (cast(c.key as int) + 0)
+  and (a.csmallint < 100)
+order by a.csmallint, a.ctinyint, a.cint
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select
+  a.*
+from
+  alltypesorc a,
+  src b,
+  src c
+where
+  a.csmallint = cast(b.key as int) and a.csmallint = (cast(c.key as int) + 0)
+  and (a.csmallint < 100)
+order by a.csmallint, a.ctinyint, a.cint
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+-19	8	626923679	NULL	-19.0	8.0	821UdmGbkEf4j	NULL	1969-12-31 15:59:46.619	1969-12-31 15:59:46.95	true	NULL
+6	8	528534767	NULL	6.0	8.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:45.459	1969-12-31 16:00:00.236	true	NULL
+NULL	9	-470743566	-1887561756	NULL	9.0	swx5K33Sm5qcKR5B	4hA4KQj2vD3fI6gX82220d	NULL	1969-12-31 16:00:07.318	true	false
+NULL	10	813877020	-1645852809	NULL	10.0	4QG23O2GKF6BUe13O7A2C	xH7445Rals48VOulSyR5F	NULL	1969-12-31 16:00:05.851	false	false
+-62	10	528534767	NULL	-62.0	10.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:52.265	1969-12-31 15:59:56.584	true	NULL
+NULL	19	312515097	1864027286	NULL	19.0	ds5YqbRvhf3Sb2	4KWs6gw7lv2WYd66P	NULL	1969-12-31 15:59:56.211	false	true
+-7	19	528534767	NULL	-7.0	19.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 16:00:13.994	1969-12-31 15:59:55.362	true	NULL
+-45	20	253665376	NULL	-45.0	20.0	1cGVWH7n1QU	NULL	1969-12-31 16:00:09.949	1969-12-31 16:00:10.979	true	NULL
+NULL	34	510824788	-1887561756	NULL	34.0	nj1bXoh6k	4hA4KQj2vD3fI6gX82220d	NULL	1969-12-31 15:59:46.017	true	false
+41	37	528534767	NULL	41.0	37.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:52.817	1969-12-31 15:59:53.672	true	NULL
+41	37	528534767	NULL	41.0	37.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:52.817	1969-12-31 15:59:53.672	true	NULL
+41	37	528534767	NULL	41.0	37.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:52.817	1969-12-31 15:59:53.672	true	NULL
+41	37	528534767	NULL	41.0	37.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:52.817	1969-12-31 15:59:53.672	true	NULL
+61	41	528534767	NULL	61.0	41.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:55.708	1969-12-31 16:00:14.412	true	NULL
+NULL	58	-144190833	-1645852809	NULL	58.0	122J3HlhqBW1D43	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:46.315	true	false
+NULL	58	-144190833	-1645852809	NULL	58.0	122J3HlhqBW1D43	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:46.315	true	false
+NULL	58	-144190833	-1645852809	NULL	58.0	122J3HlhqBW1D43	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:46.315	true	false
+NULL	58	-144190833	-1645852809	NULL	58.0	122J3HlhqBW1D43	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:46.315	true	false
+NULL	67	790444583	-1645852809	NULL	67.0	xptM81y	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:58.622	false	false
+NULL	67	790444583	-1645852809	NULL	67.0	xptM81y	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:58.622	false	false
+NULL	67	790444583	-1645852809	NULL	67.0	xptM81y	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:58.622	false	false
+NULL	67	790444583	-1645852809	NULL	67.0	xptM81y	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:58.622	false	false
+-46	72	626923679	NULL	-46.0	72.0	821UdmGbkEf4j	NULL	1969-12-31 16:00:11.411	1969-12-31 16:00:05.539	true	NULL
+-46	72	626923679	NULL	-46.0	72.0	821UdmGbkEf4j	NULL	1969-12-31 16:00:11.411	1969-12-31 16:00:05.539	true	NULL
+-46	72	626923679	NULL	-46.0	72.0	821UdmGbkEf4j	NULL	1969-12-31 16:00:11.411	1969-12-31 16:00:05.539	true	NULL
+-46	72	626923679	NULL	-46.0	72.0	821UdmGbkEf4j	NULL	1969-12-31 16:00:11.411	1969-12-31 16:00:05.539	true	NULL
+39	74	626923679	NULL	39.0	74.0	821UdmGbkEf4j	NULL	1969-12-31 16:00:10.403	1969-12-31 16:00:12.52	true	NULL
+47	74	626923679	NULL	47.0	74.0	821UdmGbkEf4j	NULL	1969-12-31 15:59:57.849	1969-12-31 15:59:57.569	true	NULL
+-22	77	528534767	NULL	-22.0	77.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:45.928	1969-12-31 15:59:43.621	true	NULL
+PREHOOK: query: -- noconditionaltask.size needs to be low enough that entire filtered table results do not fit in one task's hash table
+-- Try with dynamically partitioned hash join 
+explain
+select
+  a.*
+from
+  alltypesorc a,
+  src b,
+  src c
+where
+  a.csmallint = cast(b.key as int) and a.csmallint = (cast(c.key as int) + 0)
+  and (a.csmallint < 100)
+order by a.csmallint, a.ctinyint, a.cint
+PREHOOK: type: QUERY
+POSTHOOK: query: -- noconditionaltask.size needs to be low enough that entire filtered table results do not fit in one task's hash table
+-- Try with dynamically partitioned hash join 
+explain
+select
+  a.*
+from
+  alltypesorc a,
+  src b,
+  src c
+where
+  a.csmallint = cast(b.key as int) and a.csmallint = (cast(c.key as int) + 0)
+  and (a.csmallint < 100)
+order by a.csmallint, a.ctinyint, a.cint
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+      Edges:
+        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Map 4 (CUSTOM_SIMPLE_EDGE), Map 5 (CUSTOM_SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: ((csmallint < 100) and UDFToInteger(csmallint) is not null) (type: boolean)
+                    Statistics: Num rows: 2048 Data size: 440327 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cfloat (type: float), cdouble (type: double), cstring1 (type: string), cstring2 (type: string), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), cboolean1 (type: boolean), cboolean2 (type: boolean)
+                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
+                      Statistics: Num rows: 2048 Data size: 440327 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: UDFToInteger(_col1) (type: int)
+                        sort order: +
+                        Map-reduce partition columns: UDFToInteger(_col1) (type: int)
+                        Statistics: Num rows: 2048 Data size: 440327 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
+            Execution mode: vectorized
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: UDFToInteger(key) is not null (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
+                      Reduce Output Operator
+                        key expressions: UDFToInteger(_col0) (type: int)
+                        sort order: +
+                        Map-reduce partition columns: UDFToInteger(_col0) (type: int)
+                        Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+        Map 5 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (UDFToInteger(key) + 0) is not null (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
+                      Reduce Output Operator
+                        key expressions: (UDFToInteger(_col0) + 0) (type: int)
+                        sort order: +
+                        Map-reduce partition columns: (UDFToInteger(_col0) + 0) (type: int)
+                        Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+        Reducer 2 
+            Reduce Operator Tree:
+              Map Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                     Inner Join 0 to 2
+                keys:
+                  0 KEY.reducesinkkey0 (type: int)
+                  1 KEY.reducesinkkey0 (type: int)
+                  2 KEY.reducesinkkey0 (type: int)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
+                input vertices:
+                  1 Map 4
+                  2 Map 5
+                Statistics: Num rows: 4505 Data size: 968719 Basic stats: COMPLETE Column stats: NONE
+                HybridGraceHashJoin: true
+                Reduce Output Operator
+                  key expressions: _col1 (type: smallint), _col0 (type: tinyint), _col2 (type: int)
+                  sort order: +++
+                  Statistics: Num rows: 4505 Data size: 968719 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
+            Execution mode: vectorized
+        Reducer 3 
+            Reduce Operator Tree:
+              Select Operator
+                expressions: KEY.reducesinkkey1 (type: tinyint), KEY.reducesinkkey0 (type: smallint), KEY.reducesinkkey2 (type: int), VALUE._col0 (type: bigint), VALUE._col1 (type: float), VALUE._col2 (type: double), VALUE._col3 (type: string), VALUE._col4 (type: string), VALUE._col5 (type: timestamp), VALUE._col6 (type: timestamp), VALUE._col7 (type: boolean), VALUE._col8 (type: boolean)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
+                Statistics: Num rows: 4505 Data size: 968719 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 4505 Data size: 968719 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            Execution mode: vectorized
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select
+  a.*
+from
+  alltypesorc a,
+  src b,
+  src c
+where
+  a.csmallint = cast(b.key as int) and a.csmallint = (cast(c.key as int) + 0)
+  and (a.csmallint < 100)
+order by a.csmallint, a.ctinyint, a.cint
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select
+  a.*
+from
+  alltypesorc a,
+  src b,
+  src c
+where
+  a.csmallint = cast(b.key as int) and a.csmallint = (cast(c.key as int) + 0)
+  and (a.csmallint < 100)
+order by a.csmallint, a.ctinyint, a.cint
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+-19	8	626923679	NULL	-19.0	8.0	821UdmGbkEf4j	NULL	1969-12-31 15:59:46.619	1969-12-31 15:59:46.95	true	NULL
+6	8	528534767	NULL	6.0	8.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:45.459	1969-12-31 16:00:00.236	true	NULL
+NULL	9	-470743566	-1887561756	NULL	9.0	swx5K33Sm5qcKR5B	4hA4KQj2vD3fI6gX82220d	NULL	1969-12-31 16:00:07.318	true	false
+NULL	10	813877020	-1645852809	NULL	10.0	4QG23O2GKF6BUe13O7A2C	xH7445Rals48VOulSyR5F	NULL	1969-12-31 16:00:05.851	false	false
+-62	10	528534767	NULL	-62.0	10.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:52.265	1969-12-31 15:59:56.584	true	NULL
+NULL	19	312515097	1864027286	NULL	19.0	ds5YqbRvhf3Sb2	4KWs6gw7lv2WYd66P	NULL	1969-12-31 15:59:56.211	false	true
+-7	19	528534767	NULL	-7.0	19.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 16:00:13.994	1969-12-31 15:59:55.362	true	NULL
+-45	20	253665376	NULL	-45.0	20.0	1cGVWH7n1QU	NULL	1969-12-31 16:00:09.949	1969-12-31 16:00:10.979	true	NULL
+NULL	34	510824788	-1887561756	NULL	34.0	nj1bXoh6k	4hA4KQj2vD3fI6gX82220d	NULL	1969-12-31 15:59:46.017	true	false
+41	37	528534767	NULL	41.0	37.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:52.817	1969-12-31 15:59:53.672	true	NULL
+41	37	528534767	NULL	41.0	37.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:52.817	1969-12-31 15:59:53.672	true	NULL
+41	37	528534767	NULL	41.0	37.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:52.817	1969-12-31 15:59:53.672	true	NULL
+41	37	528534767	NULL	41.0	37.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:52.817	1969-12-31 15:59:53.672	true	NULL
+61	41	528534767	NULL	61.0	41.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:55.708	1969-12-31 16:00:14.412	true	NULL
+NULL	58	-144190833	-1645852809	NULL	58.0	122J3HlhqBW1D43	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:46.315	true	false
+NULL	58	-144190833	-1645852809	NULL	58.0	122J3HlhqBW1D43	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:46.315	true	false
+NULL	58	-144190833	-1645852809	NULL	58.0	122J3HlhqBW1D43	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:46.315	true	false
+NULL	58	-144190833	-1645852809	NULL	58.0	122J3HlhqBW1D43	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:46.315	true	false
+NULL	67	790444583	-1645852809	NULL	67.0	xptM81y	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:58.622	false	false
+NULL	67	790444583	-1645852809	NULL	67.0	xptM81y	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:58.622	false	false
+NULL	67	790444583	-1645852809	NULL	67.0	xptM81y	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:58.622	false	false
+NULL	67	790444583	-1645852809	NULL	67.0	xptM81y	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:58.622	false	false
+-46	72	626923679	NULL	-46.0	72.0	821UdmGbkEf4j	NULL	1969-12-31 16:00:11.411	1969-12-31 16:00:05.539	true	NULL
+-46	72	626923679	NULL	-46.0	72.0	821UdmGbkEf4j	NULL	1969-12-31 16:00:11.411	1969-12-31 16:00:05.539	true	NULL
+-46	72	626923679	NULL	-46.0	72.0	821UdmGbkEf4j	NULL	1969-12-31 16:00:11.411	1969-12-31 16:00:05.539	true	NULL
+-46	72	626923679	NULL	-46.0	72.0	821UdmGbkEf4j	NULL	1969-12-31 16:00:11.411	1969-12-31 16:00:05.539	true	NULL
+39	74	626923679	NULL	39.0	74.0	821UdmGbkEf4j	NULL	1969-12-31 16:00:10.403	1969-12-31 16:00:12.52	true	NULL
+47	74	626923679	NULL	47.0	74.0	821UdmGbkEf4j	NULL	1969-12-31 15:59:57.849	1969-12-31 15:59:57.569	true	NULL
+-22	77	528534767	NULL	-22.0	77.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:45.928	1969-12-31 15:59:43.621	true	NULL
+PREHOOK: query: -- Try different order of tables
+explain
+select
+  a.*
+from
+  src b,
+  alltypesorc a,
+  src c
+where
+  a.csmallint = cast(b.key as int) and a.csmallint = (cast(c.key as int) + 0)
+  and (a.csmallint < 100)
+order by a.csmallint, a.ctinyint, a.cint
+PREHOOK: type: QUERY
+POSTHOOK: query: -- Try different order of tables
+explain
+select
+  a.*
+from
+  src b,
+  alltypesorc a,
+  src c
+where
+  a.csmallint = cast(b.key as int) and a.csmallint = (cast(c.key as int) + 0)
+  and (a.csmallint < 100)
+order by a.csmallint, a.ctinyint, a.cint
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+      Edges:
+        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Map 4 (CUSTOM_SIMPLE_EDGE), Map 5 (CUSTOM_SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: ((csmallint < 100) and UDFToInteger(csmallint) is not null) (type: boolean)
+                    Statistics: Num rows: 2048 Data size: 440327 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cfloat (type: float), cdouble (type: double), cstring1 (type: string), cstring2 (type: string), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), cboolean1 (type: boolean), cboolean2 (type: boolean)
+                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
+                      Statistics: Num rows: 2048 Data size: 440327 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: UDFToInteger(_col1) (type: int)
+                        sort order: +
+                        Map-reduce partition columns: UDFToInteger(_col1) (type: int)
+                        Statistics: Num rows: 2048 Data size: 440327 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
+            Execution mode: vectorized
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: UDFToInteger(key) is not null (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
+                      Reduce Output Operator
+                        key expressions: UDFToInteger(_col0) (type: int)
+                        sort order: +
+                        Map-reduce partition columns: UDFToInteger(_col0) (type: int)
+                        Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+        Map 5 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (UDFToInteger(key) + 0) is not null (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
+                      Reduce Output Operator
+                        key expressions: (UDFToInteger(_col0) + 0) (type: int)
+                        sort order: +
+                        Map-reduce partition columns: (UDFToInteger(_col0) + 0) (type: int)
+                        Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+        Reducer 2 
+            Reduce Operator Tree:
+              Map Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                     Inner Join 0 to 2
+                keys:
+                  0 KEY.reducesinkkey0 (type: int)
+                  1 KEY.reducesinkkey0 (type: int)
+                  2 KEY.reducesinkkey0 (type: int)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
+                input vertices:
+                  1 Map 4
+                  2 Map 5
+                Statistics: Num rows: 4505 Data size: 968719 Basic stats: COMPLETE Column stats: NONE
+                HybridGraceHashJoin: true
+                Reduce Output Operator
+                  key expressions: _col1 (type: smallint), _col0 (type: tinyint), _col2 (type: int)
+                  sort order: +++
+                  Statistics: Num rows: 4505 Data size: 968719 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
+            Execution mode: vectorized
+        Reducer 3 
+            Reduce Operator Tree:
+              Select Operator
+                expressions: KEY.reducesinkkey1 (type: tinyint), KEY.reducesinkkey0 (type: smallint), KEY.reducesinkkey2 (type: int), VALUE._col0 (type: bigint), VALUE._col1 (type: float), VALUE._col2 (type: double), VALUE._col3 (type: string), VALUE._col4 (type: string), VALUE._col5 (type: timestamp), VALUE._col6 (type: timestamp), VALUE._col7 (type: boolean), VALUE._col8 (type: boolean)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
+                Statistics: Num rows: 4505 Data size: 968719 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 4505 Data size: 968719 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            Execution mode: vectorized
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select
+  a.*
+from
+  src b,
+  alltypesorc a,
+  src c
+where
+  a.csmallint = cast(b.key as int) and a.csmallint = (cast(c.key as int) + 0)
+  and (a.csmallint < 100)
+order by a.csmallint, a.ctinyint, a.cint
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select
+  a.*
+from
+  src b,
+  alltypesorc a,
+  src c
+where
+  a.csmallint = cast(b.key as int) and a.csmallint = (cast(c.key as int) + 0)
+  and (a.csmallint < 100)
+order by a.csmallint, a.ctinyint, a.cint
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+-19	8	626923679	NULL	-19.0	8.0	821UdmGbkEf4j	NULL	1969-12-31 15:59:46.619	1969-12-31 15:59:46.95	true	NULL
+6	8	528534767	NULL	6.0	8.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:45.459	1969-12-31 16:00:00.236	true	NULL
+NULL	9	-470743566	-1887561756	NULL	9.0	swx5K33Sm5qcKR5B	4hA4KQj2vD3fI6gX82220d	NULL	1969-12-31 16:00:07.318	true	false
+NULL	10	813877020	-1645852809	NULL	10.0	4QG23O2GKF6BUe13O7A2C	xH7445Rals48VOulSyR5F	NULL	1969-12-31 16:00:05.851	false	false
+-62	10	528534767	NULL	-62.0	10.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:52.265	1969-12-31 15:59:56.584	true	NULL
+NULL	19	312515097	1864027286	NULL	19.0	ds5YqbRvhf3Sb2	4KWs6gw7lv2WYd66P	NULL	1969-12-31 15:59:56.211	false	true
+-7	19	528534767	NULL	-7.0	19.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 16:00:13.994	1969-12-31 15:59:55.362	true	NULL
+-45	20	253665376	NULL	-45.0	20.0	1cGVWH7n1QU	NULL	1969-12-31 16:00:09.949	1969-12-31 16:00:10.979	true	NULL
+NULL	34	510824788	-1887561756	NULL	34.0	nj1bXoh6k	4hA4KQj2vD3fI6gX82220d	NULL	1969-12-31 15:59:46.017	true	false
+41	37	528534767	NULL	41.0	37.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:52.817	1969-12-31 15:59:53.672	true	NULL
+41	37	528534767	NULL	41.0	37.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:52.817	1969-12-31 15:59:53.672	true	NULL
+41	37	528534767	NULL	41.0	37.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:52.817	1969-12-31 15:59:53.672	true	NULL
+41	37	528534767	NULL	41.0	37.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:52.817	1969-12-31 15:59:53.672	true	NULL
+61	41	528534767	NULL	61.0	41.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:55.708	1969-12-31 16:00:14.412	true	NULL
+NULL	58	-144190833	-1645852809	NULL	58.0	122J3HlhqBW1D43	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:46.315	true	false
+NULL	58	-144190833	-1645852809	NULL	58.0	122J3HlhqBW1D43	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:46.315	true	false
+NULL	58	-144190833	-1645852809	NULL	58.0	122J3HlhqBW1D43	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:46.315	true	false
+NULL	58	-144190833	-1645852809	NULL	58.0	122J3HlhqBW1D43	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:46.315	true	false
+NULL	67	790444583	-1645852809	NULL	67.0	xptM81y	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:58.622	false	false
+NULL	67	790444583	-1645852809	NULL	67.0	xptM81y	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:58.622	false	false
+NULL	67	790444583	-1645852809	NULL	67.0	xptM81y	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:58.622	false	false
+NULL	67	790444583	-1645852809	NULL	67.0	xptM81y	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:58.622	false	false
+-46	72	626923679	NULL	-46.0	72.0	821UdmGbkEf4j	NULL	1969-12-31 16:00:11.411	1969-12-31 16:00:05.539	true	NULL
+-46	72	626923679	NULL	-46.0	72.0	821UdmGbkEf4j	NULL	1969-12-31 16:00:11.411	1969-12-31 16:00:05.539	true	NULL
+-46	72	626923679	NULL	-46.0	72.0	821UdmGbkEf4j	NULL	1969-12-31 16:00:11.411	1969-12-31 16:00:05.539	true	NULL
+-46	72	626923679	NULL	-46.0	72.0	821UdmGbkEf4j	NULL	1969-12-31 16:00:11.411	1969-12-31 16:00:05.539	true	NULL
+39	74	626923679	NULL	39.0	74.0	821UdmGbkEf4j	NULL	1969-12-31 16:00:10.403	1969-12-31 16:00:12.52	true	NULL
+47	74	626923679	NULL	47.0	74.0	821UdmGbkEf4j	NULL	1969-12-31 15:59:57.849	1969-12-31 15:59:57.569	true	NULL
+-22	77	528534767	NULL	-22.0	77.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:45.928	1969-12-31 15:59:43.621	true	NULL


[29/43] hive git commit: HIVE-10799. Refactor the SearchArgumentFactory to remove the AST-specific factory. (omalley reviewed by prasanth_j)

Posted by xu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/c178a6e9/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestRecordReaderImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestRecordReaderImpl.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestRecordReaderImpl.java
index 957f54e..7957cb4 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestRecordReaderImpl.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestRecordReaderImpl.java
@@ -42,7 +42,9 @@ import org.apache.hadoop.hive.ql.io.orc.RecordReaderImpl.Location;
 import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument.TruthValue;
 import org.apache.hadoop.hive.ql.io.sarg.TestSearchArgumentImpl;
+import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.junit.Test;
 import org.mockito.MockSettings;
@@ -351,14 +353,14 @@ public class TestRecordReaderImpl {
         RecordReaderImpl.evaluatePredicateProto(createBooleanStats(10, 0), pred, null));
 
     pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.BOOLEAN, "x", "true", null);
+        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.BOOLEAN, "x", true, null);
     assertEquals(TruthValue.YES_NO,
         RecordReaderImpl.evaluatePredicateProto(createBooleanStats(10, 10), pred, null));
     assertEquals(TruthValue.NO,
         RecordReaderImpl.evaluatePredicateProto(createBooleanStats(10, 0), pred, null));
 
     pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.BOOLEAN, "x", "hello", null);
+        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.BOOLEAN, "x", false, null);
     assertEquals(TruthValue.NO,
         RecordReaderImpl.evaluatePredicateProto(createBooleanStats(10, 10), pred, null));
     assertEquals(TruthValue.YES_NO,
@@ -368,7 +370,7 @@ public class TestRecordReaderImpl {
   @Test
   public void testPredEvalWithIntStats() throws Exception {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.INTEGER, "x", 15L, null);
+        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.LONG, "x", 15L, null);
     assertEquals(TruthValue.YES_NO,
         RecordReaderImpl.evaluatePredicateProto(createIntStats(10, 100), pred, null));
 
@@ -390,7 +392,7 @@ public class TestRecordReaderImpl {
         RecordReaderImpl.evaluatePredicateProto(createIntStats(10, 100), pred, null));
 
     pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.DECIMAL, "x", HiveDecimal.create(15), null);
+        PredicateLeaf.Type.DECIMAL, "x", new HiveDecimalWritable("15"), null);
     assertEquals(TruthValue.YES_NO,
         RecordReaderImpl.evaluatePredicateProto(createIntStats(10, 100), pred, null));
 
@@ -403,7 +405,7 @@ public class TestRecordReaderImpl {
   @Test
   public void testPredEvalWithDoubleStats() throws Exception {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.INTEGER, "x", 15L, null);
+        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.LONG, "x", 15L, null);
     assertEquals(TruthValue.YES_NO,
         RecordReaderImpl.evaluatePredicateProto(createDoubleStats(10.0, 100.0), pred, null));
 
@@ -425,7 +427,7 @@ public class TestRecordReaderImpl {
         RecordReaderImpl.evaluatePredicateProto(createDoubleStats(10.0, 100.0), pred, null));
 
     pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.DECIMAL, "x", HiveDecimal.create(15), null);
+        PredicateLeaf.Type.DECIMAL, "x", new HiveDecimalWritable("15"), null);
     assertEquals(TruthValue.YES_NO,
         RecordReaderImpl.evaluatePredicateProto(createDoubleStats(10.0, 100.0), pred, null));
 
@@ -461,10 +463,10 @@ public class TestRecordReaderImpl {
     pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
         PredicateLeaf.Type.DATE, "x", new DateWritable(100).get(), null);
     assertEquals(TruthValue.YES_NO,
-        RecordReaderImpl.evaluatePredicateProto(createStringStats("10", "1000"), pred, null));
+        RecordReaderImpl.evaluatePredicateProto(createDateStats(10, 1000), pred, null));
 
     pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.DECIMAL, "x", HiveDecimal.create(100), null);
+        PredicateLeaf.Type.DECIMAL, "x", new HiveDecimalWritable("100"), null);
     assertEquals(TruthValue.YES_NO,
         RecordReaderImpl.evaluatePredicateProto(createStringStats("10", "1000"), pred, null));
 
@@ -477,7 +479,7 @@ public class TestRecordReaderImpl {
   @Test
   public void testPredEvalWithDateStats() throws Exception {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.INTEGER, "x", 15L, null);
+        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.LONG, "x", 15L, null);
     // Date to Integer conversion is not possible.
     assertEquals(TruthValue.YES_NO,
         RecordReaderImpl.evaluatePredicateProto(createDateStats(10, 100), pred, null));
@@ -530,7 +532,7 @@ public class TestRecordReaderImpl {
 
     // Date to Decimal conversion is also not possible.
     pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.DECIMAL, "x", HiveDecimal.create(15), null);
+        PredicateLeaf.Type.DECIMAL, "x", new HiveDecimalWritable("15"), null);
     assertEquals(TruthValue.YES_NO,
         RecordReaderImpl.evaluatePredicateProto(createDateStats(10, 100), pred, null));
 
@@ -548,7 +550,7 @@ public class TestRecordReaderImpl {
   @Test
   public void testPredEvalWithDecimalStats() throws Exception {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.INTEGER, "x", 15L, null);
+        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.INTEGER, "x", 15, null);
     assertEquals(TruthValue.YES_NO,
         RecordReaderImpl.evaluatePredicateProto(createDecimalStats("10.0", "100.0"), pred, null));
 
@@ -570,7 +572,7 @@ public class TestRecordReaderImpl {
         RecordReaderImpl.evaluatePredicateProto(createDecimalStats("10.0", "100.0"), pred, null));
 
     pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.DECIMAL, "x", HiveDecimal.create(15), null);
+        PredicateLeaf.Type.DECIMAL, "x", new HiveDecimalWritable("15"), null);
     assertEquals(TruthValue.YES_NO,
         RecordReaderImpl.evaluatePredicateProto(createDecimalStats("10.0", "100.0"), pred, null));
 
@@ -588,7 +590,7 @@ public class TestRecordReaderImpl {
   @Test
   public void testPredEvalWithTimestampStats() throws Exception {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.INTEGER, "x", 15L, null);
+        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.INTEGER, "x", 15, null);
     assertEquals(TruthValue.YES_NO,
         RecordReaderImpl.evaluatePredicateProto(createTimestampStats(10, 100), pred, null));
 
@@ -618,7 +620,7 @@ public class TestRecordReaderImpl {
             100 * 24L * 60L * 60L * 1000L), pred, null));
 
     pred = TestSearchArgumentImpl.createPredicateLeaf(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
-        PredicateLeaf.Type.DECIMAL, "x", HiveDecimal.create(15), null);
+        PredicateLeaf.Type.DECIMAL, "x", new HiveDecimalWritable("15"), null);
     assertEquals(TruthValue.NO,
         RecordReaderImpl.evaluatePredicateProto(createTimestampStats(10, 100), pred, null));
     assertEquals(TruthValue.YES_NO,
@@ -636,7 +638,7 @@ public class TestRecordReaderImpl {
   public void testEquals() throws Exception {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
         (PredicateLeaf.Operator.EQUALS, PredicateLeaf.Type.INTEGER,
-            "x", 15L, null);
+            "x", 15, null);
     assertEquals(TruthValue.NO_NULL,
         RecordReaderImpl.evaluatePredicateProto(createIntStats(20L, 30L), pred, null));
     assertEquals(TruthValue.YES_NO_NULL,
@@ -655,7 +657,7 @@ public class TestRecordReaderImpl {
   public void testNullSafeEquals() throws Exception {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
         (PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.INTEGER,
-            "x", 15L, null);
+            "x", 15, null);
     assertEquals(TruthValue.NO,
         RecordReaderImpl.evaluatePredicateProto(createIntStats(20L, 30L), pred, null));
     assertEquals(TruthValue.YES_NO,
@@ -674,7 +676,7 @@ public class TestRecordReaderImpl {
   public void testLessThan() throws Exception {
     PredicateLeaf lessThan = TestSearchArgumentImpl.createPredicateLeaf
         (PredicateLeaf.Operator.LESS_THAN, PredicateLeaf.Type.INTEGER,
-            "x", 15L, null);
+            "x", 15, null);
     assertEquals(TruthValue.NO_NULL,
         RecordReaderImpl.evaluatePredicateProto(createIntStats(20L, 30L), lessThan, null));
     assertEquals(TruthValue.NO_NULL,
@@ -691,7 +693,7 @@ public class TestRecordReaderImpl {
   public void testLessThanEquals() throws Exception {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
         (PredicateLeaf.Operator.LESS_THAN_EQUALS, PredicateLeaf.Type.INTEGER,
-            "x", 15L, null);
+            "x", 15, null);
     assertEquals(TruthValue.NO_NULL,
         RecordReaderImpl.evaluatePredicateProto(createIntStats(20L, 30L), pred, null));
     assertEquals(TruthValue.YES_NO_NULL,
@@ -707,8 +709,8 @@ public class TestRecordReaderImpl {
   @Test
   public void testIn() throws Exception {
     List<Object> args = new ArrayList<Object>();
-    args.add(10L);
-    args.add(20L);
+    args.add(10);
+    args.add(20);
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
         (PredicateLeaf.Operator.IN, PredicateLeaf.Type.INTEGER,
             "x", null, args);
@@ -725,8 +727,8 @@ public class TestRecordReaderImpl {
   @Test
   public void testBetween() throws Exception {
     List<Object> args = new ArrayList<Object>();
-    args.add(10L);
-    args.add(20L);
+    args.add(10);
+    args.add(20);
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
         (PredicateLeaf.Operator.BETWEEN, PredicateLeaf.Type.INTEGER,
             "x", null, args);
@@ -1274,7 +1276,7 @@ public class TestRecordReaderImpl {
   @Test
   public void testIntNullSafeEqualsBloomFilter() throws Exception {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.INTEGER, "x", 15L, null);
+        PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.LONG, "x", 15L, null);
     BloomFilterIO bf = new BloomFilterIO(10000);
     for (int i = 20; i < 1000; i++) {
       bf.addLong(i);
@@ -1289,7 +1291,7 @@ public class TestRecordReaderImpl {
   @Test
   public void testIntEqualsBloomFilter() throws Exception {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.EQUALS, PredicateLeaf.Type.INTEGER, "x", 15L, null);
+        PredicateLeaf.Operator.EQUALS, PredicateLeaf.Type.LONG, "x", 15L, null);
     BloomFilterIO bf = new BloomFilterIO(10000);
     for (int i = 20; i < 1000; i++) {
       bf.addLong(i);
@@ -1539,7 +1541,7 @@ public class TestRecordReaderImpl {
   public void testDecimalNullSafeEqualsBloomFilter() throws Exception {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
         PredicateLeaf.Operator.NULL_SAFE_EQUALS, PredicateLeaf.Type.DECIMAL, "x",
-        HiveDecimal.create(15),
+        new HiveDecimalWritable("15"),
         null);
     BloomFilterIO bf = new BloomFilterIO(10000);
     for (int i = 20; i < 1000; i++) {
@@ -1555,7 +1557,8 @@ public class TestRecordReaderImpl {
   @Test
   public void testDecimalEqualsBloomFilter() throws Exception {
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf(
-        PredicateLeaf.Operator.EQUALS, PredicateLeaf.Type.DECIMAL, "x", HiveDecimal.create(15),
+        PredicateLeaf.Operator.EQUALS, PredicateLeaf.Type.DECIMAL, "x",
+        new HiveDecimalWritable("15"),
         null);
     BloomFilterIO bf = new BloomFilterIO(10000);
     for (int i = 20; i < 1000; i++) {
@@ -1571,8 +1574,8 @@ public class TestRecordReaderImpl {
   @Test
   public void testDecimalInBloomFilter() throws Exception {
     List<Object> args = new ArrayList<Object>();
-    args.add(HiveDecimal.create(15));
-    args.add(HiveDecimal.create(19));
+    args.add(new HiveDecimalWritable("15"));
+    args.add(new HiveDecimalWritable("19"));
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
         (PredicateLeaf.Operator.IN, PredicateLeaf.Type.DECIMAL,
             "x", null, args);
@@ -1593,9 +1596,9 @@ public class TestRecordReaderImpl {
   @Test
   public void testNullsInBloomFilter() throws Exception {
     List<Object> args = new ArrayList<Object>();
-    args.add(HiveDecimal.create(15));
+    args.add(new HiveDecimalWritable("15"));
     args.add(null);
-    args.add(HiveDecimal.create(19));
+    args.add(new HiveDecimalWritable("19"));
     PredicateLeaf pred = TestSearchArgumentImpl.createPredicateLeaf
         (PredicateLeaf.Operator.IN, PredicateLeaf.Type.DECIMAL,
             "x", null, args);

http://git-wip-us.apache.org/repos/asf/hive/blob/c178a6e9/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestParquetRecordReaderWrapper.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestParquetRecordReaderWrapper.java b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestParquetRecordReaderWrapper.java
new file mode 100644
index 0000000..87dd344
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/parquet/TestParquetRecordReaderWrapper.java
@@ -0,0 +1,155 @@
+/**
+ * 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.io.parquet;
+
+import static junit.framework.Assert.assertEquals;
+
+import org.apache.hadoop.hive.common.type.HiveChar;
+import org.apache.hadoop.hive.common.type.HiveVarchar;
+import org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument.TruthValue;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.junit.Test;
+
+import java.sql.Date;
+
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+
+/**
+ * These tests test the conversion to Parquet's sarg implementation.
+ */
+public class TestParquetRecordReaderWrapper {
+
+  private static TruthValue[] values(TruthValue... vals) {
+    return vals;
+  }
+
+  @Test
+  public void testBuilder() throws Exception {
+     SearchArgument sarg = SearchArgumentFactory.newBuilder()
+        .startNot()
+        .startOr()
+        .isNull("x", PredicateLeaf.Type.INTEGER)
+        .between("y", PredicateLeaf.Type.INTEGER, 10, 20)
+        .in("z", PredicateLeaf.Type.INTEGER, 1, 2, 3)
+        .nullSafeEquals("a", PredicateLeaf.Type.STRING, "stinger")
+        .end()
+        .end()
+        .build();
+
+    FilterPredicate p = ParquetRecordReaderWrapper.toFilterPredicate(sarg);
+    String expected =
+      "and(and(and(not(eq(x, null)), not(and(lt(y, 20), not(lteq(y, 10))))), not(or(or(eq(z, 1), " +
+        "eq(z, 2)), eq(z, 3)))), not(eq(a, Binary{\"stinger\"})))";
+    assertEquals(expected, p.toString());
+  }
+
+  @Test
+  public void testBuilderComplexTypes() throws Exception {
+    SearchArgument sarg =
+        SearchArgumentFactory.newBuilder()
+            .startAnd()
+            .lessThan("x", PredicateLeaf.Type.DATE,
+                Date.valueOf("1970-1-11"))
+            .lessThanEquals("y", PredicateLeaf.Type.STRING,
+                new HiveChar("hi", 10).toString())
+            .equals("z", PredicateLeaf.Type.DECIMAL, new HiveDecimalWritable("1.0"))
+            .end()
+            .build();
+    assertEquals("lteq(y, Binary{\"hi        \"})",
+        ParquetRecordReaderWrapper.toFilterPredicate(sarg).toString());
+
+    sarg = SearchArgumentFactory.newBuilder()
+        .startNot()
+        .startOr()
+        .isNull("x", PredicateLeaf.Type.INTEGER)
+        .between("y", PredicateLeaf.Type.DECIMAL,
+            new HiveDecimalWritable("10"), new HiveDecimalWritable("20.0"))
+        .in("z", PredicateLeaf.Type.INTEGER, 1, 2, 3)
+        .nullSafeEquals("a", PredicateLeaf.Type.STRING,
+            new HiveVarchar("stinger", 100).toString())
+        .end()
+        .end()
+        .build();
+
+    FilterPredicate p = ParquetRecordReaderWrapper.toFilterPredicate(sarg);
+    String expected =
+        "and(and(not(eq(x, null)), not(or(or(eq(z, 1), eq(z, 2)), eq(z, 3)))), " +
+        "not(eq(a, Binary{\"stinger\"})))";
+    assertEquals(expected, p.toString());
+  }
+
+  @Test
+  public void testBuilderComplexTypes2() throws Exception {
+    SearchArgument sarg =
+        SearchArgumentFactory.newBuilder()
+            .startAnd()
+            .lessThan("x", PredicateLeaf.Type.DATE, Date.valueOf("2005-3-12"))
+            .lessThanEquals("y", PredicateLeaf.Type.STRING,
+                new HiveChar("hi", 10).toString())
+            .equals("z", PredicateLeaf.Type.DECIMAL,
+                new HiveDecimalWritable("1.0"))
+            .end()
+            .build();
+    assertEquals("lteq(y, Binary{\"hi        \"})",
+        ParquetRecordReaderWrapper.toFilterPredicate(sarg).toString());
+
+    sarg = SearchArgumentFactory.newBuilder()
+        .startNot()
+        .startOr()
+        .isNull("x", PredicateLeaf.Type.INTEGER)
+        .between("y", PredicateLeaf.Type.DECIMAL, new HiveDecimalWritable("10"),
+            new HiveDecimalWritable("20.0"))
+        .in("z", PredicateLeaf.Type.INTEGER, 1, 2, 3)
+        .nullSafeEquals("a", PredicateLeaf.Type.STRING,
+            new HiveVarchar("stinger", 100).toString())
+        .end()
+        .end()
+        .build();
+
+    FilterPredicate p = ParquetRecordReaderWrapper.toFilterPredicate(sarg);
+    String expected = "and(and(not(eq(x, null)), not(or(or(eq(z, 1), eq(z, 2)), eq(z, 3)))), " +
+        "not(eq(a, Binary{\"stinger\"})))";
+    assertEquals(expected, p.toString());
+  }
+
+  @Test
+  public void testBuilderFloat() throws Exception {
+    SearchArgument sarg =
+        SearchArgumentFactory.newBuilder()
+            .startAnd()
+            .lessThan("x", PredicateLeaf.Type.INTEGER, new Integer((short) 22))
+            .lessThan("x1", PredicateLeaf.Type.INTEGER, new Integer(22))
+            .lessThanEquals("y", PredicateLeaf.Type.STRING,
+                new HiveChar("hi", 10).toString())
+            .equals("z", PredicateLeaf.Type.FLOAT, new Double(0.22))
+            .equals("z1", PredicateLeaf.Type.FLOAT, new Double(0.22))
+            .end()
+            .build();
+
+    FilterPredicate p = ParquetRecordReaderWrapper.toFilterPredicate(sarg);
+    String expected = "and(and(and(and(lt(x, 22), lt(x1, 22))," +
+        " lteq(y, Binary{\"hi        \"})), eq(z, " +
+        "0.22)), eq(z1, 0.22))";
+    assertEquals(expected, p.toString());
+  }
+}


[02/43] hive git commit: HIVE-11244: Beeline prompt info improvement for cluster mode(Nemon Lou, reviewed by Ferdinand Xu)

Posted by xu...@apache.org.
HIVE-11244: Beeline prompt info improvement for cluster mode(Nemon Lou, reviewed by Ferdinand Xu)


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

Branch: refs/heads/spark
Commit: 09448652d254b211eb81efbf163936df591e18b6
Parents: 0ad4f71
Author: Ferdinand Xu <ch...@intel.com>
Authored: Mon Jul 20 21:26:53 2015 -0400
Committer: Ferdinand Xu <ch...@intel.com>
Committed: Mon Jul 20 21:26:53 2015 -0400

----------------------------------------------------------------------
 beeline/src/java/org/apache/hive/beeline/BeeLine.java       | 5 +++--
 beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java   | 9 +++++++++
 .../java/org/apache/hive/beeline/DatabaseConnection.java    | 9 +++++++++
 beeline/src/main/resources/BeeLine.properties               | 2 ++
 jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java      | 4 ++++
 5 files changed, 27 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/09448652/beeline/src/java/org/apache/hive/beeline/BeeLine.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/BeeLine.java b/beeline/src/java/org/apache/hive/beeline/BeeLine.java
index c760c94..1519619 100644
--- a/beeline/src/java/org/apache/hive/beeline/BeeLine.java
+++ b/beeline/src/java/org/apache/hive/beeline/BeeLine.java
@@ -1233,8 +1233,9 @@ public class BeeLine implements Closeable {
       return "beeline> ";
     } else {
       String printClosed = getDatabaseConnection().isClosed() ? " (closed)" : "";
-      return getPrompt(getDatabaseConnections().getIndex()
-          + ": " + getDatabaseConnection().getUrl()) + printClosed + "> ";
+      String url =  getOpts().getShowConnectedUrl() ? getDatabaseConnection().getConnectedUrl()
+          : getDatabaseConnection().getUrl();
+      return getPrompt(getDatabaseConnections().getIndex() + ": " + url) + printClosed + "> ";
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/09448652/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java b/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java
index d8b8b78..a31c49c 100644
--- a/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java
+++ b/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java
@@ -80,6 +80,7 @@ class BeeLineOpts implements Completer {
   private String outputFormat = "table";
   private boolean trimScripts = true;
   private boolean allowMultiLineCommand = true;
+  private boolean showConnectedUrl = false;
 
   //This can be set for old behavior of nulls printed as empty strings
   private boolean nullEmptyString = false;
@@ -434,6 +435,14 @@ class BeeLineOpts implements Completer {
     return autosave;
   }
 
+  public boolean getShowConnectedUrl() {
+    return showConnectedUrl;
+  }
+
+  public void setShowConnectedUrl(boolean showConnectedUrl) {
+    this.showConnectedUrl = showConnectedUrl;
+  }
+
   public void setOutputFormat(String outputFormat) {
     if(outputFormat.equalsIgnoreCase("csv") || outputFormat.equalsIgnoreCase("tsv")) {
       beeLine.info("Format " + outputFormat + " is deprecated, please use " + outputFormat + "2");

http://git-wip-us.apache.org/repos/asf/hive/blob/09448652/beeline/src/java/org/apache/hive/beeline/DatabaseConnection.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/DatabaseConnection.java b/beeline/src/java/org/apache/hive/beeline/DatabaseConnection.java
index 02b2eec..e349a49 100644
--- a/beeline/src/java/org/apache/hive/beeline/DatabaseConnection.java
+++ b/beeline/src/java/org/apache/hive/beeline/DatabaseConnection.java
@@ -38,6 +38,8 @@ import java.util.Properties;
 import java.util.Set;
 import java.util.TreeSet;
 
+import org.apache.hive.jdbc.HiveConnection;
+
 import jline.console.completer.ArgumentCompleter;
 import jline.console.completer.Completer;
 
@@ -264,6 +266,13 @@ class DatabaseConnection {
     return url;
   }
 
+  public String getConnectedUrl() {
+    if (connection instanceof HiveConnection) {
+      return ((HiveConnection) connection).getConnectedUrl();
+    }
+    return getUrl();
+  }
+
   Completer getSQLCompleter() {
     return sqlCompleter;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/09448652/beeline/src/main/resources/BeeLine.properties
----------------------------------------------------------------------
diff --git a/beeline/src/main/resources/BeeLine.properties b/beeline/src/main/resources/BeeLine.properties
index 756c334..a118c09 100644
--- a/beeline/src/main/resources/BeeLine.properties
+++ b/beeline/src/main/resources/BeeLine.properties
@@ -178,6 +178,8 @@ cmd-usage: Usage: java org.apache.hive.cli.beeline.BeeLine \n \
 \  --nullemptystring=[true/false]  set to true to get historic behavior of printing null as empty string\n \
 \  --addlocaldriverjar=DRIVERJARNAME Add driver jar file in the beeline client side\n \
 \  --addlocaldrivername=DRIVERNAME Add drvier name needs to be supported in the beeline client side\n \
+\  --showConnectedUrl=[true/false] Prompt HiveServer2's URI to which this beeline connected.\n \
+\                                  Only works for HiveServer2 cluster mode.\n \
 \  --help                          display this message
 
 

http://git-wip-us.apache.org/repos/asf/hive/blob/09448652/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java
----------------------------------------------------------------------
diff --git a/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java b/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java
index 959822e..bb2b695 100644
--- a/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java
+++ b/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java
@@ -240,6 +240,10 @@ public class HiveConnection implements java.sql.Connection {
     }
   }
 
+  public String getConnectedUrl() {
+    return jdbcUriString;
+  }
+
   private String getServerHttpUrl(boolean useSsl) {
     // Create the http/https url
     // JDBC driver will set up an https url if ssl is enabled, otherwise http


[08/43] hive git commit: HIVE-11303: Getting Tez LimitExceededException after dag execution on large query (Jason Dere, reviewed by Vikram Dixit)

Posted by xu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/04d54f61/ql/src/test/results/clientpositive/tez/tez_dynpart_hashjoin_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/tez_dynpart_hashjoin_1.q.out b/ql/src/test/results/clientpositive/tez/tez_dynpart_hashjoin_1.q.out
new file mode 100644
index 0000000..8a46807
--- /dev/null
+++ b/ql/src/test/results/clientpositive/tez/tez_dynpart_hashjoin_1.q.out
@@ -0,0 +1,791 @@
+PREHOOK: query: -- First try with regular mergejoin
+explain
+select
+  *
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+order by a.cint
+PREHOOK: type: QUERY
+POSTHOOK: query: -- First try with regular mergejoin
+explain
+select
+  *
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+order by a.cint
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+      Edges:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (cint BETWEEN 1000000 AND 3000000 and cint is not null) (type: boolean)
+                    Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cfloat (type: float), cdouble (type: double), cstring1 (type: string), cstring2 (type: string), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), cboolean1 (type: boolean), cboolean2 (type: boolean)
+                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
+                      Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col2 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col2 (type: int)
+                        Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: ((cbigint is not null and cint BETWEEN 1000000 AND 3000000) and cint is not null) (type: boolean)
+                    Statistics: Num rows: 1536 Data size: 330245 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cfloat (type: float), cdouble (type: double), cstring1 (type: string), cstring2 (type: string), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), cboolean1 (type: boolean), cboolean2 (type: boolean)
+                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
+                      Statistics: Num rows: 1536 Data size: 330245 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col2 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col2 (type: int)
+                        Statistics: Num rows: 1536 Data size: 330245 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
+        Reducer 2 
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col2 (type: int)
+                  1 _col2 (type: int)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23
+                Statistics: Num rows: 3379 Data size: 726540 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col2 (type: int)
+                  sort order: +
+                  Statistics: Num rows: 3379 Data size: 726540 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean), _col12 (type: tinyint), _col13 (type: smallint), _col14 (type: int), _col15 (type: bigint), _col16 (type: float), _col17 (type: double), _col18 (type: string), _col19 (type: string), _col20 (type: timestamp), _col21 (type: timestamp), _col22 (type: boolean), _col23 (type: boolean)
+        Reducer 3 
+            Reduce Operator Tree:
+              Select Operator
+                expressions: VALUE._col0 (type: tinyint), VALUE._col1 (type: smallint), KEY.reducesinkkey0 (type: int), VALUE._col2 (type: bigint), VALUE._col3 (type: float), VALUE._col4 (type: double), VALUE._col5 (type: string), VALUE._col6 (type: string), VALUE._col7 (type: timestamp), VALUE._col8 (type: timestamp), VALUE._col9 (type: boolean), VALUE._col10 (type: boolean), VALUE._col11 (type: tinyint), VALUE._col12 (type: smallint), VALUE._col13 (type: int), VALUE._col14 (type: bigint), VALUE._col15 (type: float), VALUE._col16 (type: double), VALUE._col17 (type: string), VALUE._col18 (type: string), VALUE._col19 (type: timestamp), VALUE._col20 (type: timestamp), VALUE._col21 (type: boolean), VALUE._col22 (type: boolean)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23
+                Statistics: Num rows: 3379 Data size: 726540 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 3379 Data size: 726540 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select
+  *
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+order by a.cint
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+POSTHOOK: query: select
+  *
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+order by a.cint
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+11	NULL	1000828	1531084669	11.0	NULL	wM316f6NqGIkoP388j3F6	poWQQo3Upvt3Wh	1969-12-31 16:00:02.351	NULL	false	true	11	NULL	1000828	1531084669	11.0	NULL	wM316f6NqGIkoP388j3F6	poWQQo3Upvt3Wh	1969-12-31 16:00:02.351	NULL	false	true
+NULL	-3799	1248059	1864027286	NULL	-3799.0	Uhps6mMh3IfHB3j7yH62K	4KWs6gw7lv2WYd66P	NULL	1969-12-31 15:59:54.622	false	true	NULL	-3799	1248059	1864027286	NULL	-3799.0	Uhps6mMh3IfHB3j7yH62K	4KWs6gw7lv2WYd66P	NULL	1969-12-31 15:59:54.622	false	true
+NULL	10782	1286921	1864027286	NULL	10782.0	ODLrXI8882q8LS8	4KWs6gw7lv2WYd66P	NULL	1969-12-31 15:59:52.138	true	true	NULL	10782	1286921	1864027286	NULL	10782.0	ODLrXI8882q8LS8	4KWs6gw7lv2WYd66P	NULL	1969-12-31 15:59:52.138	true	true
+NULL	-13036	1288927	-1645852809	NULL	-13036.0	yinBY725P7V2	xH7445Rals48VOulSyR5F	NULL	1969-12-31 16:00:00.763	true	false	NULL	-13036	1288927	-1645852809	NULL	-13036.0	yinBY725P7V2	xH7445Rals48VOulSyR5F	NULL	1969-12-31 16:00:00.763	true	false
+11	NULL	1310786	-413875656	11.0	NULL	W0rvA4H1xn0xMG4uk0	8yVVjG	1969-12-31 16:00:02.351	NULL	false	true	11	NULL	1310786	-413875656	11.0	NULL	W0rvA4H1xn0xMG4uk0	8yVVjG	1969-12-31 16:00:02.351	NULL	false	true
+-51	NULL	2089466	-240556350	-51.0	NULL	cXX24dH7tblSj46j2g	C31eea0wrHHqvj	1969-12-31 16:00:08.451	NULL	true	true	-51	NULL	2089466	-240556350	-51.0	NULL	cXX24dH7tblSj46j2g	C31eea0wrHHqvj	1969-12-31 16:00:08.451	NULL	true	true
+NULL	-8915	2101183	1864027286	NULL	-8915.0	x7By66525	4KWs6gw7lv2WYd66P	NULL	1969-12-31 16:00:05.831	false	true	NULL	-8915	2101183	1864027286	NULL	-8915.0	x7By66525	4KWs6gw7lv2WYd66P	NULL	1969-12-31 16:00:05.831	false	true
+8	NULL	2229621	-381406148	8.0	NULL	q7onkS7QRPh5ghOK	oKb0bi	1969-12-31 16:00:15.892	NULL	true	false	8	NULL	2229621	-381406148	8.0	NULL	q7onkS7QRPh5ghOK	oKb0bi	1969-12-31 16:00:15.892	NULL	true	false
+8	NULL	2433892	-1611863517	8.0	NULL	674ILv3V2TxFqXP6wSbL	VLprkK2XfX	1969-12-31 16:00:15.892	NULL	false	true	8	NULL	2433892	-1611863517	8.0	NULL	674ILv3V2TxFqXP6wSbL	VLprkK2XfX	1969-12-31 16:00:15.892	NULL	false	true
+-51	NULL	2949963	-1580871111	-51.0	NULL	0K68k3bdl7jO7	TPPAu	1969-12-31 16:00:08.451	NULL	true	false	-51	NULL	2949963	-1580871111	-51.0	NULL	0K68k3bdl7jO7	TPPAu	1969-12-31 16:00:08.451	NULL	true	false
+PREHOOK: query: explain
+select
+  count(*)
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select
+  count(*)
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+      Edges:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (cint BETWEEN 1000000 AND 3000000 and cint is not null) (type: boolean)
+                    Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: cint (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 3072 Data size: 660491 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: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: ((cbigint is not null and cint BETWEEN 1000000 AND 3000000) and cint is not null) (type: boolean)
+                    Statistics: Num rows: 1536 Data size: 330245 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: cint (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1536 Data size: 330245 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: 1536 Data size: 330245 Basic stats: COMPLETE Column stats: NONE
+        Reducer 2 
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: int)
+                  1 _col0 (type: int)
+                Statistics: Num rows: 3379 Data size: 726540 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: count()
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col0 (type: bigint)
+        Reducer 3 
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select
+  count(*)
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+POSTHOOK: query: select
+  count(*)
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+10
+PREHOOK: query: explain
+select
+  a.csmallint, count(*) c1
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+group by a.csmallint
+order by c1
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select
+  a.csmallint, count(*) c1
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+group by a.csmallint
+order by c1
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+      Edges:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+        Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (cint BETWEEN 1000000 AND 3000000 and cint is not null) (type: boolean)
+                    Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: csmallint (type: smallint), cint (type: int)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col1 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col1 (type: int)
+                        Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: smallint)
+        Map 5 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: ((cbigint is not null and cint BETWEEN 1000000 AND 3000000) and cint is not null) (type: boolean)
+                    Statistics: Num rows: 1536 Data size: 330245 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: cint (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1536 Data size: 330245 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: 1536 Data size: 330245 Basic stats: COMPLETE Column stats: NONE
+        Reducer 2 
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col1 (type: int)
+                  1 _col0 (type: int)
+                outputColumnNames: _col0
+                Statistics: Num rows: 3379 Data size: 726540 Basic stats: COMPLETE Column stats: NONE
+                Group By Operator
+                  aggregations: count()
+                  keys: _col0 (type: smallint)
+                  mode: hash
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 3379 Data size: 726540 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: smallint)
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: smallint)
+                    Statistics: Num rows: 3379 Data size: 726540 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col1 (type: bigint)
+        Reducer 3 
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                keys: KEY._col0 (type: smallint)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1689 Data size: 363162 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col1 (type: bigint)
+                  sort order: +
+                  Statistics: Num rows: 1689 Data size: 363162 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col0 (type: smallint)
+        Reducer 4 
+            Reduce Operator Tree:
+              Select Operator
+                expressions: VALUE._col0 (type: smallint), KEY.reducesinkkey0 (type: bigint)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1689 Data size: 363162 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1689 Data size: 363162 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select
+  a.csmallint, count(*) c1
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+group by a.csmallint
+order by c1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+POSTHOOK: query: select
+  a.csmallint, count(*) c1
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+group by a.csmallint
+order by c1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+10782	1
+-3799	1
+-8915	1
+-13036	1
+NULL	6
+PREHOOK: query: -- Try with dynamically partitioned hashjoin
+explain
+select
+  *
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+order by a.cint
+PREHOOK: type: QUERY
+POSTHOOK: query: -- Try with dynamically partitioned hashjoin
+explain
+select
+  *
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+order by a.cint
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+      Edges:
+        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Map 4 (CUSTOM_SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (cint BETWEEN 1000000 AND 3000000 and cint is not null) (type: boolean)
+                    Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cfloat (type: float), cdouble (type: double), cstring1 (type: string), cstring2 (type: string), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), cboolean1 (type: boolean), cboolean2 (type: boolean)
+                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
+                      Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col2 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col2 (type: int)
+                        Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: ((cbigint is not null and cint BETWEEN 1000000 AND 3000000) and cint is not null) (type: boolean)
+                    Statistics: Num rows: 1536 Data size: 330245 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cfloat (type: float), cdouble (type: double), cstring1 (type: string), cstring2 (type: string), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), cboolean1 (type: boolean), cboolean2 (type: boolean)
+                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
+                      Statistics: Num rows: 1536 Data size: 330245 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col2 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col2 (type: int)
+                        Statistics: Num rows: 1536 Data size: 330245 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
+        Reducer 2 
+            Reduce Operator Tree:
+              Map Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 KEY.reducesinkkey0 (type: int)
+                  1 KEY.reducesinkkey0 (type: int)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23
+                input vertices:
+                  1 Map 4
+                Statistics: Num rows: 3379 Data size: 726540 Basic stats: COMPLETE Column stats: NONE
+                HybridGraceHashJoin: true
+                Reduce Output Operator
+                  key expressions: _col2 (type: int)
+                  sort order: +
+                  Statistics: Num rows: 3379 Data size: 726540 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean), _col12 (type: tinyint), _col13 (type: smallint), _col14 (type: int), _col15 (type: bigint), _col16 (type: float), _col17 (type: double), _col18 (type: string), _col19 (type: string), _col20 (type: timestamp), _col21 (type: timestamp), _col22 (type: boolean), _col23 (type: boolean)
+        Reducer 3 
+            Reduce Operator Tree:
+              Select Operator
+                expressions: VALUE._col0 (type: tinyint), VALUE._col1 (type: smallint), KEY.reducesinkkey0 (type: int), VALUE._col2 (type: bigint), VALUE._col3 (type: float), VALUE._col4 (type: double), VALUE._col5 (type: string), VALUE._col6 (type: string), VALUE._col7 (type: timestamp), VALUE._col8 (type: timestamp), VALUE._col9 (type: boolean), VALUE._col10 (type: boolean), VALUE._col11 (type: tinyint), VALUE._col12 (type: smallint), VALUE._col13 (type: int), VALUE._col14 (type: bigint), VALUE._col15 (type: float), VALUE._col16 (type: double), VALUE._col17 (type: string), VALUE._col18 (type: string), VALUE._col19 (type: timestamp), VALUE._col20 (type: timestamp), VALUE._col21 (type: boolean), VALUE._col22 (type: boolean)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23
+                Statistics: Num rows: 3379 Data size: 726540 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 3379 Data size: 726540 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select
+  *
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+order by a.cint
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+POSTHOOK: query: select
+  *
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+order by a.cint
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+11	NULL	1000828	1531084669	11.0	NULL	wM316f6NqGIkoP388j3F6	poWQQo3Upvt3Wh	1969-12-31 16:00:02.351	NULL	false	true	11	NULL	1000828	1531084669	11.0	NULL	wM316f6NqGIkoP388j3F6	poWQQo3Upvt3Wh	1969-12-31 16:00:02.351	NULL	false	true
+NULL	-3799	1248059	1864027286	NULL	-3799.0	Uhps6mMh3IfHB3j7yH62K	4KWs6gw7lv2WYd66P	NULL	1969-12-31 15:59:54.622	false	true	NULL	-3799	1248059	1864027286	NULL	-3799.0	Uhps6mMh3IfHB3j7yH62K	4KWs6gw7lv2WYd66P	NULL	1969-12-31 15:59:54.622	false	true
+NULL	10782	1286921	1864027286	NULL	10782.0	ODLrXI8882q8LS8	4KWs6gw7lv2WYd66P	NULL	1969-12-31 15:59:52.138	true	true	NULL	10782	1286921	1864027286	NULL	10782.0	ODLrXI8882q8LS8	4KWs6gw7lv2WYd66P	NULL	1969-12-31 15:59:52.138	true	true
+NULL	-13036	1288927	-1645852809	NULL	-13036.0	yinBY725P7V2	xH7445Rals48VOulSyR5F	NULL	1969-12-31 16:00:00.763	true	false	NULL	-13036	1288927	-1645852809	NULL	-13036.0	yinBY725P7V2	xH7445Rals48VOulSyR5F	NULL	1969-12-31 16:00:00.763	true	false
+11	NULL	1310786	-413875656	11.0	NULL	W0rvA4H1xn0xMG4uk0	8yVVjG	1969-12-31 16:00:02.351	NULL	false	true	11	NULL	1310786	-413875656	11.0	NULL	W0rvA4H1xn0xMG4uk0	8yVVjG	1969-12-31 16:00:02.351	NULL	false	true
+-51	NULL	2089466	-240556350	-51.0	NULL	cXX24dH7tblSj46j2g	C31eea0wrHHqvj	1969-12-31 16:00:08.451	NULL	true	true	-51	NULL	2089466	-240556350	-51.0	NULL	cXX24dH7tblSj46j2g	C31eea0wrHHqvj	1969-12-31 16:00:08.451	NULL	true	true
+NULL	-8915	2101183	1864027286	NULL	-8915.0	x7By66525	4KWs6gw7lv2WYd66P	NULL	1969-12-31 16:00:05.831	false	true	NULL	-8915	2101183	1864027286	NULL	-8915.0	x7By66525	4KWs6gw7lv2WYd66P	NULL	1969-12-31 16:00:05.831	false	true
+8	NULL	2229621	-381406148	8.0	NULL	q7onkS7QRPh5ghOK	oKb0bi	1969-12-31 16:00:15.892	NULL	true	false	8	NULL	2229621	-381406148	8.0	NULL	q7onkS7QRPh5ghOK	oKb0bi	1969-12-31 16:00:15.892	NULL	true	false
+8	NULL	2433892	-1611863517	8.0	NULL	674ILv3V2TxFqXP6wSbL	VLprkK2XfX	1969-12-31 16:00:15.892	NULL	false	true	8	NULL	2433892	-1611863517	8.0	NULL	674ILv3V2TxFqXP6wSbL	VLprkK2XfX	1969-12-31 16:00:15.892	NULL	false	true
+-51	NULL	2949963	-1580871111	-51.0	NULL	0K68k3bdl7jO7	TPPAu	1969-12-31 16:00:08.451	NULL	true	false	-51	NULL	2949963	-1580871111	-51.0	NULL	0K68k3bdl7jO7	TPPAu	1969-12-31 16:00:08.451	NULL	true	false
+PREHOOK: query: explain
+select
+  count(*)
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select
+  count(*)
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+      Edges:
+        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Map 4 (CUSTOM_SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (cint BETWEEN 1000000 AND 3000000 and cint is not null) (type: boolean)
+                    Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: cint (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 3072 Data size: 660491 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: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: ((cbigint is not null and cint BETWEEN 1000000 AND 3000000) and cint is not null) (type: boolean)
+                    Statistics: Num rows: 1536 Data size: 330245 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: cint (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1536 Data size: 330245 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: 1536 Data size: 330245 Basic stats: COMPLETE Column stats: NONE
+        Reducer 2 
+            Reduce Operator Tree:
+              Map Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 KEY.reducesinkkey0 (type: int)
+                  1 KEY.reducesinkkey0 (type: int)
+                input vertices:
+                  1 Map 4
+                Statistics: Num rows: 3379 Data size: 726540 Basic stats: COMPLETE Column stats: NONE
+                HybridGraceHashJoin: true
+                Group By Operator
+                  aggregations: count()
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col0 (type: bigint)
+        Reducer 3 
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                mode: mergepartial
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select
+  count(*)
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+POSTHOOK: query: select
+  count(*)
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+10
+PREHOOK: query: explain
+select
+  a.csmallint, count(*) c1
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+group by a.csmallint
+order by c1
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select
+  a.csmallint, count(*) c1
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+group by a.csmallint
+order by c1
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+      Edges:
+        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Map 5 (CUSTOM_SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+        Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (cint BETWEEN 1000000 AND 3000000 and cint is not null) (type: boolean)
+                    Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: csmallint (type: smallint), cint (type: int)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col1 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col1 (type: int)
+                        Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: smallint)
+        Map 5 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: ((cbigint is not null and cint BETWEEN 1000000 AND 3000000) and cint is not null) (type: boolean)
+                    Statistics: Num rows: 1536 Data size: 330245 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: cint (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1536 Data size: 330245 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: 1536 Data size: 330245 Basic stats: COMPLETE Column stats: NONE
+        Reducer 2 
+            Reduce Operator Tree:
+              Map Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 KEY.reducesinkkey0 (type: int)
+                  1 KEY.reducesinkkey0 (type: int)
+                outputColumnNames: _col0
+                input vertices:
+                  1 Map 5
+                Statistics: Num rows: 3379 Data size: 726540 Basic stats: COMPLETE Column stats: NONE
+                HybridGraceHashJoin: true
+                Group By Operator
+                  aggregations: count()
+                  keys: _col0 (type: smallint)
+                  mode: hash
+                  outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 3379 Data size: 726540 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    key expressions: _col0 (type: smallint)
+                    sort order: +
+                    Map-reduce partition columns: _col0 (type: smallint)
+                    Statistics: Num rows: 3379 Data size: 726540 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col1 (type: bigint)
+        Reducer 3 
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: count(VALUE._col0)
+                keys: KEY._col0 (type: smallint)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1689 Data size: 363162 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col1 (type: bigint)
+                  sort order: +
+                  Statistics: Num rows: 1689 Data size: 363162 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col0 (type: smallint)
+        Reducer 4 
+            Reduce Operator Tree:
+              Select Operator
+                expressions: VALUE._col0 (type: smallint), KEY.reducesinkkey0 (type: bigint)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 1689 Data size: 363162 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1689 Data size: 363162 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select
+  a.csmallint, count(*) c1
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+group by a.csmallint
+order by c1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+POSTHOOK: query: select
+  a.csmallint, count(*) c1
+from alltypesorc a join alltypesorc b on a.cint = b.cint
+where
+  a.cint between 1000000 and 3000000 and b.cbigint is not null
+group by a.csmallint
+order by c1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+10782	1
+-3799	1
+-8915	1
+-13036	1
+NULL	6

http://git-wip-us.apache.org/repos/asf/hive/blob/04d54f61/ql/src/test/results/clientpositive/tez/tez_dynpart_hashjoin_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/tez_dynpart_hashjoin_2.q.out b/ql/src/test/results/clientpositive/tez/tez_dynpart_hashjoin_2.q.out
new file mode 100644
index 0000000..8f43b26
--- /dev/null
+++ b/ql/src/test/results/clientpositive/tez/tez_dynpart_hashjoin_2.q.out
@@ -0,0 +1,564 @@
+PREHOOK: query: -- Multiple tables, and change the order of the big table (alltypesorc)
+-- First try with regular mergejoin
+explain
+select
+  a.*
+from
+  alltypesorc a,
+  src b,
+  src c
+where
+  a.csmallint = cast(b.key as int) and a.csmallint = (cast(c.key as int) + 0)
+  and (a.csmallint < 100)
+order by a.csmallint, a.ctinyint, a.cint
+PREHOOK: type: QUERY
+POSTHOOK: query: -- Multiple tables, and change the order of the big table (alltypesorc)
+-- First try with regular mergejoin
+explain
+select
+  a.*
+from
+  alltypesorc a,
+  src b,
+  src c
+where
+  a.csmallint = cast(b.key as int) and a.csmallint = (cast(c.key as int) + 0)
+  and (a.csmallint < 100)
+order by a.csmallint, a.ctinyint, a.cint
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+      Edges:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: ((csmallint < 100) and UDFToInteger(csmallint) is not null) (type: boolean)
+                    Statistics: Num rows: 2048 Data size: 440327 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cfloat (type: float), cdouble (type: double), cstring1 (type: string), cstring2 (type: string), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), cboolean1 (type: boolean), cboolean2 (type: boolean)
+                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
+                      Statistics: Num rows: 2048 Data size: 440327 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: UDFToInteger(_col1) (type: int)
+                        sort order: +
+                        Map-reduce partition columns: UDFToInteger(_col1) (type: int)
+                        Statistics: Num rows: 2048 Data size: 440327 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: UDFToInteger(key) is not null (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
+                      Reduce Output Operator
+                        key expressions: UDFToInteger(_col0) (type: int)
+                        sort order: +
+                        Map-reduce partition columns: UDFToInteger(_col0) (type: int)
+                        Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+        Map 5 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (UDFToInteger(key) + 0) is not null (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
+                      Reduce Output Operator
+                        key expressions: (UDFToInteger(_col0) + 0) (type: int)
+                        sort order: +
+                        Map-reduce partition columns: (UDFToInteger(_col0) + 0) (type: int)
+                        Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+        Reducer 2 
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                     Inner Join 0 to 2
+                keys:
+                  0 UDFToInteger(_col1) (type: int)
+                  1 UDFToInteger(_col0) (type: int)
+                  2 (UDFToInteger(_col0) + 0) (type: int)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
+                Statistics: Num rows: 4505 Data size: 968719 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col1 (type: smallint), _col0 (type: tinyint), _col2 (type: int)
+                  sort order: +++
+                  Statistics: Num rows: 4505 Data size: 968719 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
+        Reducer 3 
+            Reduce Operator Tree:
+              Select Operator
+                expressions: KEY.reducesinkkey1 (type: tinyint), KEY.reducesinkkey0 (type: smallint), KEY.reducesinkkey2 (type: int), VALUE._col0 (type: bigint), VALUE._col1 (type: float), VALUE._col2 (type: double), VALUE._col3 (type: string), VALUE._col4 (type: string), VALUE._col5 (type: timestamp), VALUE._col6 (type: timestamp), VALUE._col7 (type: boolean), VALUE._col8 (type: boolean)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
+                Statistics: Num rows: 4505 Data size: 968719 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 4505 Data size: 968719 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select
+  a.*
+from
+  alltypesorc a,
+  src b,
+  src c
+where
+  a.csmallint = cast(b.key as int) and a.csmallint = (cast(c.key as int) + 0)
+  and (a.csmallint < 100)
+order by a.csmallint, a.ctinyint, a.cint
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select
+  a.*
+from
+  alltypesorc a,
+  src b,
+  src c
+where
+  a.csmallint = cast(b.key as int) and a.csmallint = (cast(c.key as int) + 0)
+  and (a.csmallint < 100)
+order by a.csmallint, a.ctinyint, a.cint
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+-19	8	626923679	NULL	-19.0	8.0	821UdmGbkEf4j	NULL	1969-12-31 15:59:46.619	1969-12-31 15:59:46.95	true	NULL
+6	8	528534767	NULL	6.0	8.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:45.459	1969-12-31 16:00:00.236	true	NULL
+NULL	9	-470743566	-1887561756	NULL	9.0	swx5K33Sm5qcKR5B	4hA4KQj2vD3fI6gX82220d	NULL	1969-12-31 16:00:07.318	true	false
+NULL	10	813877020	-1645852809	NULL	10.0	4QG23O2GKF6BUe13O7A2C	xH7445Rals48VOulSyR5F	NULL	1969-12-31 16:00:05.851	false	false
+-62	10	528534767	NULL	-62.0	10.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:52.265	1969-12-31 15:59:56.584	true	NULL
+NULL	19	312515097	1864027286	NULL	19.0	ds5YqbRvhf3Sb2	4KWs6gw7lv2WYd66P	NULL	1969-12-31 15:59:56.211	false	true
+-7	19	528534767	NULL	-7.0	19.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 16:00:13.994	1969-12-31 15:59:55.362	true	NULL
+-45	20	253665376	NULL	-45.0	20.0	1cGVWH7n1QU	NULL	1969-12-31 16:00:09.949	1969-12-31 16:00:10.979	true	NULL
+NULL	34	510824788	-1887561756	NULL	34.0	nj1bXoh6k	4hA4KQj2vD3fI6gX82220d	NULL	1969-12-31 15:59:46.017	true	false
+41	37	528534767	NULL	41.0	37.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:52.817	1969-12-31 15:59:53.672	true	NULL
+41	37	528534767	NULL	41.0	37.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:52.817	1969-12-31 15:59:53.672	true	NULL
+41	37	528534767	NULL	41.0	37.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:52.817	1969-12-31 15:59:53.672	true	NULL
+41	37	528534767	NULL	41.0	37.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:52.817	1969-12-31 15:59:53.672	true	NULL
+61	41	528534767	NULL	61.0	41.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:55.708	1969-12-31 16:00:14.412	true	NULL
+NULL	58	-144190833	-1645852809	NULL	58.0	122J3HlhqBW1D43	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:46.315	true	false
+NULL	58	-144190833	-1645852809	NULL	58.0	122J3HlhqBW1D43	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:46.315	true	false
+NULL	58	-144190833	-1645852809	NULL	58.0	122J3HlhqBW1D43	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:46.315	true	false
+NULL	58	-144190833	-1645852809	NULL	58.0	122J3HlhqBW1D43	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:46.315	true	false
+NULL	67	790444583	-1645852809	NULL	67.0	xptM81y	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:58.622	false	false
+NULL	67	790444583	-1645852809	NULL	67.0	xptM81y	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:58.622	false	false
+NULL	67	790444583	-1645852809	NULL	67.0	xptM81y	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:58.622	false	false
+NULL	67	790444583	-1645852809	NULL	67.0	xptM81y	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:58.622	false	false
+-46	72	626923679	NULL	-46.0	72.0	821UdmGbkEf4j	NULL	1969-12-31 16:00:11.411	1969-12-31 16:00:05.539	true	NULL
+-46	72	626923679	NULL	-46.0	72.0	821UdmGbkEf4j	NULL	1969-12-31 16:00:11.411	1969-12-31 16:00:05.539	true	NULL
+-46	72	626923679	NULL	-46.0	72.0	821UdmGbkEf4j	NULL	1969-12-31 16:00:11.411	1969-12-31 16:00:05.539	true	NULL
+-46	72	626923679	NULL	-46.0	72.0	821UdmGbkEf4j	NULL	1969-12-31 16:00:11.411	1969-12-31 16:00:05.539	true	NULL
+39	74	626923679	NULL	39.0	74.0	821UdmGbkEf4j	NULL	1969-12-31 16:00:10.403	1969-12-31 16:00:12.52	true	NULL
+47	74	626923679	NULL	47.0	74.0	821UdmGbkEf4j	NULL	1969-12-31 15:59:57.849	1969-12-31 15:59:57.569	true	NULL
+-22	77	528534767	NULL	-22.0	77.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:45.928	1969-12-31 15:59:43.621	true	NULL
+PREHOOK: query: -- noconditionaltask.size needs to be low enough that entire filtered table results do not fit in one task's hash table
+-- Try with dynamically partitioned hash join 
+explain
+select
+  a.*
+from
+  alltypesorc a,
+  src b,
+  src c
+where
+  a.csmallint = cast(b.key as int) and a.csmallint = (cast(c.key as int) + 0)
+  and (a.csmallint < 100)
+order by a.csmallint, a.ctinyint, a.cint
+PREHOOK: type: QUERY
+POSTHOOK: query: -- noconditionaltask.size needs to be low enough that entire filtered table results do not fit in one task's hash table
+-- Try with dynamically partitioned hash join 
+explain
+select
+  a.*
+from
+  alltypesorc a,
+  src b,
+  src c
+where
+  a.csmallint = cast(b.key as int) and a.csmallint = (cast(c.key as int) + 0)
+  and (a.csmallint < 100)
+order by a.csmallint, a.ctinyint, a.cint
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+      Edges:
+        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Map 4 (CUSTOM_SIMPLE_EDGE), Map 5 (CUSTOM_SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: ((csmallint < 100) and UDFToInteger(csmallint) is not null) (type: boolean)
+                    Statistics: Num rows: 2048 Data size: 440327 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cfloat (type: float), cdouble (type: double), cstring1 (type: string), cstring2 (type: string), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), cboolean1 (type: boolean), cboolean2 (type: boolean)
+                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
+                      Statistics: Num rows: 2048 Data size: 440327 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: UDFToInteger(_col1) (type: int)
+                        sort order: +
+                        Map-reduce partition columns: UDFToInteger(_col1) (type: int)
+                        Statistics: Num rows: 2048 Data size: 440327 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: UDFToInteger(key) is not null (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
+                      Reduce Output Operator
+                        key expressions: UDFToInteger(_col0) (type: int)
+                        sort order: +
+                        Map-reduce partition columns: UDFToInteger(_col0) (type: int)
+                        Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+        Map 5 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (UDFToInteger(key) + 0) is not null (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
+                      Reduce Output Operator
+                        key expressions: (UDFToInteger(_col0) + 0) (type: int)
+                        sort order: +
+                        Map-reduce partition columns: (UDFToInteger(_col0) + 0) (type: int)
+                        Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+        Reducer 2 
+            Reduce Operator Tree:
+              Map Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                     Inner Join 0 to 2
+                keys:
+                  0 KEY.reducesinkkey0 (type: int)
+                  1 KEY.reducesinkkey0 (type: int)
+                  2 KEY.reducesinkkey0 (type: int)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
+                input vertices:
+                  1 Map 4
+                  2 Map 5
+                Statistics: Num rows: 4505 Data size: 968719 Basic stats: COMPLETE Column stats: NONE
+                HybridGraceHashJoin: true
+                Reduce Output Operator
+                  key expressions: _col1 (type: smallint), _col0 (type: tinyint), _col2 (type: int)
+                  sort order: +++
+                  Statistics: Num rows: 4505 Data size: 968719 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
+        Reducer 3 
+            Reduce Operator Tree:
+              Select Operator
+                expressions: KEY.reducesinkkey1 (type: tinyint), KEY.reducesinkkey0 (type: smallint), KEY.reducesinkkey2 (type: int), VALUE._col0 (type: bigint), VALUE._col1 (type: float), VALUE._col2 (type: double), VALUE._col3 (type: string), VALUE._col4 (type: string), VALUE._col5 (type: timestamp), VALUE._col6 (type: timestamp), VALUE._col7 (type: boolean), VALUE._col8 (type: boolean)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
+                Statistics: Num rows: 4505 Data size: 968719 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 4505 Data size: 968719 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select
+  a.*
+from
+  alltypesorc a,
+  src b,
+  src c
+where
+  a.csmallint = cast(b.key as int) and a.csmallint = (cast(c.key as int) + 0)
+  and (a.csmallint < 100)
+order by a.csmallint, a.ctinyint, a.cint
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select
+  a.*
+from
+  alltypesorc a,
+  src b,
+  src c
+where
+  a.csmallint = cast(b.key as int) and a.csmallint = (cast(c.key as int) + 0)
+  and (a.csmallint < 100)
+order by a.csmallint, a.ctinyint, a.cint
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+-19	8	626923679	NULL	-19.0	8.0	821UdmGbkEf4j	NULL	1969-12-31 15:59:46.619	1969-12-31 15:59:46.95	true	NULL
+6	8	528534767	NULL	6.0	8.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:45.459	1969-12-31 16:00:00.236	true	NULL
+NULL	9	-470743566	-1887561756	NULL	9.0	swx5K33Sm5qcKR5B	4hA4KQj2vD3fI6gX82220d	NULL	1969-12-31 16:00:07.318	true	false
+NULL	10	813877020	-1645852809	NULL	10.0	4QG23O2GKF6BUe13O7A2C	xH7445Rals48VOulSyR5F	NULL	1969-12-31 16:00:05.851	false	false
+-62	10	528534767	NULL	-62.0	10.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:52.265	1969-12-31 15:59:56.584	true	NULL
+NULL	19	312515097	1864027286	NULL	19.0	ds5YqbRvhf3Sb2	4KWs6gw7lv2WYd66P	NULL	1969-12-31 15:59:56.211	false	true
+-7	19	528534767	NULL	-7.0	19.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 16:00:13.994	1969-12-31 15:59:55.362	true	NULL
+-45	20	253665376	NULL	-45.0	20.0	1cGVWH7n1QU	NULL	1969-12-31 16:00:09.949	1969-12-31 16:00:10.979	true	NULL
+NULL	34	510824788	-1887561756	NULL	34.0	nj1bXoh6k	4hA4KQj2vD3fI6gX82220d	NULL	1969-12-31 15:59:46.017	true	false
+41	37	528534767	NULL	41.0	37.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:52.817	1969-12-31 15:59:53.672	true	NULL
+41	37	528534767	NULL	41.0	37.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:52.817	1969-12-31 15:59:53.672	true	NULL
+41	37	528534767	NULL	41.0	37.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:52.817	1969-12-31 15:59:53.672	true	NULL
+41	37	528534767	NULL	41.0	37.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:52.817	1969-12-31 15:59:53.672	true	NULL
+61	41	528534767	NULL	61.0	41.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:55.708	1969-12-31 16:00:14.412	true	NULL
+NULL	58	-144190833	-1645852809	NULL	58.0	122J3HlhqBW1D43	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:46.315	true	false
+NULL	58	-144190833	-1645852809	NULL	58.0	122J3HlhqBW1D43	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:46.315	true	false
+NULL	58	-144190833	-1645852809	NULL	58.0	122J3HlhqBW1D43	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:46.315	true	false
+NULL	58	-144190833	-1645852809	NULL	58.0	122J3HlhqBW1D43	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:46.315	true	false
+NULL	67	790444583	-1645852809	NULL	67.0	xptM81y	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:58.622	false	false
+NULL	67	790444583	-1645852809	NULL	67.0	xptM81y	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:58.622	false	false
+NULL	67	790444583	-1645852809	NULL	67.0	xptM81y	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:58.622	false	false
+NULL	67	790444583	-1645852809	NULL	67.0	xptM81y	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:58.622	false	false
+-46	72	626923679	NULL	-46.0	72.0	821UdmGbkEf4j	NULL	1969-12-31 16:00:11.411	1969-12-31 16:00:05.539	true	NULL
+-46	72	626923679	NULL	-46.0	72.0	821UdmGbkEf4j	NULL	1969-12-31 16:00:11.411	1969-12-31 16:00:05.539	true	NULL
+-46	72	626923679	NULL	-46.0	72.0	821UdmGbkEf4j	NULL	1969-12-31 16:00:11.411	1969-12-31 16:00:05.539	true	NULL
+-46	72	626923679	NULL	-46.0	72.0	821UdmGbkEf4j	NULL	1969-12-31 16:00:11.411	1969-12-31 16:00:05.539	true	NULL
+39	74	626923679	NULL	39.0	74.0	821UdmGbkEf4j	NULL	1969-12-31 16:00:10.403	1969-12-31 16:00:12.52	true	NULL
+47	74	626923679	NULL	47.0	74.0	821UdmGbkEf4j	NULL	1969-12-31 15:59:57.849	1969-12-31 15:59:57.569	true	NULL
+-22	77	528534767	NULL	-22.0	77.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:45.928	1969-12-31 15:59:43.621	true	NULL
+PREHOOK: query: -- Try different order of tables
+explain
+select
+  a.*
+from
+  src b,
+  alltypesorc a,
+  src c
+where
+  a.csmallint = cast(b.key as int) and a.csmallint = (cast(c.key as int) + 0)
+  and (a.csmallint < 100)
+order by a.csmallint, a.ctinyint, a.cint
+PREHOOK: type: QUERY
+POSTHOOK: query: -- Try different order of tables
+explain
+select
+  a.*
+from
+  src b,
+  alltypesorc a,
+  src c
+where
+  a.csmallint = cast(b.key as int) and a.csmallint = (cast(c.key as int) + 0)
+  and (a.csmallint < 100)
+order by a.csmallint, a.ctinyint, a.cint
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+      Edges:
+        Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Map 4 (CUSTOM_SIMPLE_EDGE), Map 5 (CUSTOM_SIMPLE_EDGE)
+        Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: a
+                  Statistics: Num rows: 12288 Data size: 2641964 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: ((csmallint < 100) and UDFToInteger(csmallint) is not null) (type: boolean)
+                    Statistics: Num rows: 2048 Data size: 440327 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: ctinyint (type: tinyint), csmallint (type: smallint), cint (type: int), cbigint (type: bigint), cfloat (type: float), cdouble (type: double), cstring1 (type: string), cstring2 (type: string), ctimestamp1 (type: timestamp), ctimestamp2 (type: timestamp), cboolean1 (type: boolean), cboolean2 (type: boolean)
+                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
+                      Statistics: Num rows: 2048 Data size: 440327 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: UDFToInteger(_col1) (type: int)
+                        sort order: +
+                        Map-reduce partition columns: UDFToInteger(_col1) (type: int)
+                        Statistics: Num rows: 2048 Data size: 440327 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: tinyint), _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
+        Map 4 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: UDFToInteger(key) is not null (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
+                      Reduce Output Operator
+                        key expressions: UDFToInteger(_col0) (type: int)
+                        sort order: +
+                        Map-reduce partition columns: UDFToInteger(_col0) (type: int)
+                        Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+        Map 5 
+            Map Operator Tree:
+                TableScan
+                  alias: b
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: (UDFToInteger(key) + 0) is not null (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
+                      Reduce Output Operator
+                        key expressions: (UDFToInteger(_col0) + 0) (type: int)
+                        sort order: +
+                        Map-reduce partition columns: (UDFToInteger(_col0) + 0) (type: int)
+                        Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+        Reducer 2 
+            Reduce Operator Tree:
+              Map Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                     Inner Join 0 to 2
+                keys:
+                  0 KEY.reducesinkkey0 (type: int)
+                  1 KEY.reducesinkkey0 (type: int)
+                  2 KEY.reducesinkkey0 (type: int)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
+                input vertices:
+                  1 Map 4
+                  2 Map 5
+                Statistics: Num rows: 4505 Data size: 968719 Basic stats: COMPLETE Column stats: NONE
+                HybridGraceHashJoin: true
+                Reduce Output Operator
+                  key expressions: _col1 (type: smallint), _col0 (type: tinyint), _col2 (type: int)
+                  sort order: +++
+                  Statistics: Num rows: 4505 Data size: 968719 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col3 (type: bigint), _col4 (type: float), _col5 (type: double), _col6 (type: string), _col7 (type: string), _col8 (type: timestamp), _col9 (type: timestamp), _col10 (type: boolean), _col11 (type: boolean)
+        Reducer 3 
+            Reduce Operator Tree:
+              Select Operator
+                expressions: KEY.reducesinkkey1 (type: tinyint), KEY.reducesinkkey0 (type: smallint), KEY.reducesinkkey2 (type: int), VALUE._col0 (type: bigint), VALUE._col1 (type: float), VALUE._col2 (type: double), VALUE._col3 (type: string), VALUE._col4 (type: string), VALUE._col5 (type: timestamp), VALUE._col6 (type: timestamp), VALUE._col7 (type: boolean), VALUE._col8 (type: boolean)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
+                Statistics: Num rows: 4505 Data size: 968719 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 4505 Data size: 968719 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select
+  a.*
+from
+  src b,
+  alltypesorc a,
+  src c
+where
+  a.csmallint = cast(b.key as int) and a.csmallint = (cast(c.key as int) + 0)
+  and (a.csmallint < 100)
+order by a.csmallint, a.ctinyint, a.cint
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select
+  a.*
+from
+  src b,
+  alltypesorc a,
+  src c
+where
+  a.csmallint = cast(b.key as int) and a.csmallint = (cast(c.key as int) + 0)
+  and (a.csmallint < 100)
+order by a.csmallint, a.ctinyint, a.cint
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+-19	8	626923679	NULL	-19.0	8.0	821UdmGbkEf4j	NULL	1969-12-31 15:59:46.619	1969-12-31 15:59:46.95	true	NULL
+6	8	528534767	NULL	6.0	8.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:45.459	1969-12-31 16:00:00.236	true	NULL
+NULL	9	-470743566	-1887561756	NULL	9.0	swx5K33Sm5qcKR5B	4hA4KQj2vD3fI6gX82220d	NULL	1969-12-31 16:00:07.318	true	false
+NULL	10	813877020	-1645852809	NULL	10.0	4QG23O2GKF6BUe13O7A2C	xH7445Rals48VOulSyR5F	NULL	1969-12-31 16:00:05.851	false	false
+-62	10	528534767	NULL	-62.0	10.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:52.265	1969-12-31 15:59:56.584	true	NULL
+NULL	19	312515097	1864027286	NULL	19.0	ds5YqbRvhf3Sb2	4KWs6gw7lv2WYd66P	NULL	1969-12-31 15:59:56.211	false	true
+-7	19	528534767	NULL	-7.0	19.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 16:00:13.994	1969-12-31 15:59:55.362	true	NULL
+-45	20	253665376	NULL	-45.0	20.0	1cGVWH7n1QU	NULL	1969-12-31 16:00:09.949	1969-12-31 16:00:10.979	true	NULL
+NULL	34	510824788	-1887561756	NULL	34.0	nj1bXoh6k	4hA4KQj2vD3fI6gX82220d	NULL	1969-12-31 15:59:46.017	true	false
+41	37	528534767	NULL	41.0	37.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:52.817	1969-12-31 15:59:53.672	true	NULL
+41	37	528534767	NULL	41.0	37.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:52.817	1969-12-31 15:59:53.672	true	NULL
+41	37	528534767	NULL	41.0	37.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:52.817	1969-12-31 15:59:53.672	true	NULL
+41	37	528534767	NULL	41.0	37.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:52.817	1969-12-31 15:59:53.672	true	NULL
+61	41	528534767	NULL	61.0	41.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:55.708	1969-12-31 16:00:14.412	true	NULL
+NULL	58	-144190833	-1645852809	NULL	58.0	122J3HlhqBW1D43	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:46.315	true	false
+NULL	58	-144190833	-1645852809	NULL	58.0	122J3HlhqBW1D43	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:46.315	true	false
+NULL	58	-144190833	-1645852809	NULL	58.0	122J3HlhqBW1D43	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:46.315	true	false
+NULL	58	-144190833	-1645852809	NULL	58.0	122J3HlhqBW1D43	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:46.315	true	false
+NULL	67	790444583	-1645852809	NULL	67.0	xptM81y	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:58.622	false	false
+NULL	67	790444583	-1645852809	NULL	67.0	xptM81y	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:58.622	false	false
+NULL	67	790444583	-1645852809	NULL	67.0	xptM81y	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:58.622	false	false
+NULL	67	790444583	-1645852809	NULL	67.0	xptM81y	xH7445Rals48VOulSyR5F	NULL	1969-12-31 15:59:58.622	false	false
+-46	72	626923679	NULL	-46.0	72.0	821UdmGbkEf4j	NULL	1969-12-31 16:00:11.411	1969-12-31 16:00:05.539	true	NULL
+-46	72	626923679	NULL	-46.0	72.0	821UdmGbkEf4j	NULL	1969-12-31 16:00:11.411	1969-12-31 16:00:05.539	true	NULL
+-46	72	626923679	NULL	-46.0	72.0	821UdmGbkEf4j	NULL	1969-12-31 16:00:11.411	1969-12-31 16:00:05.539	true	NULL
+-46	72	626923679	NULL	-46.0	72.0	821UdmGbkEf4j	NULL	1969-12-31 16:00:11.411	1969-12-31 16:00:05.539	true	NULL
+39	74	626923679	NULL	39.0	74.0	821UdmGbkEf4j	NULL	1969-12-31 16:00:10.403	1969-12-31 16:00:12.52	true	NULL
+47	74	626923679	NULL	47.0	74.0	821UdmGbkEf4j	NULL	1969-12-31 15:59:57.849	1969-12-31 15:59:57.569	true	NULL
+-22	77	528534767	NULL	-22.0	77.0	cvLH6Eat2yFsyy7p	NULL	1969-12-31 15:59:45.928	1969-12-31 15:59:43.621	true	NULL


[34/43] hive git commit: HIVE-11390: CBO (Calcite Return Path): Fix table alias propagation for windowing (Jesus Camacho Rodriguez, reviewed by Pengcheng Xiong)

Posted by xu...@apache.org.
HIVE-11390: CBO (Calcite Return Path): Fix table alias propagation for windowing (Jesus Camacho Rodriguez, reviewed by Pengcheng Xiong)


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

Branch: refs/heads/spark
Commit: 70b56e372d1bf792a5b13c217e0004785a84a10f
Parents: cd2b499
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Tue Jul 28 20:22:26 2015 +0200
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Wed Jul 29 11:52:54 2015 +0200

----------------------------------------------------------------------
 .../hive/ql/optimizer/calcite/translator/HiveOpConverter.java      | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/70b56e37/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverter.java
index c54a601..169c351 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverter.java
@@ -605,7 +605,7 @@ public class HiveOpConverter {
     WindowingComponentizer groups = new WindowingComponentizer(wSpec);
     RowResolver rr = new RowResolver();
     for (ColumnInfo ci : input.getSchema().getSignature()) {
-      rr.put(ci.getTabAlias(), ci.getInternalName(), ci);
+      rr.put(inputOpAf.tabAlias, ci.getInternalName(), ci);
     }
 
     while (groups.hasNext()) {


[14/43] hive git commit: HIVE-11254 Process result sets returned by a stored procedure (Dmitry Tolpeko via gates)

Posted by xu...@apache.org.
HIVE-11254 Process result sets returned by a stored procedure (Dmitry Tolpeko via gates)


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

Branch: refs/heads/spark
Commit: 2240dbd6dfddf3f14fb1538bb765833b3fdea898
Parents: 57242e3
Author: Alan Gates <ga...@hortonworks.com>
Authored: Wed Jul 22 10:26:55 2015 -0700
Committer: Alan Gates <ga...@hortonworks.com>
Committed: Wed Jul 22 10:26:55 2015 -0700

----------------------------------------------------------------------
 .../antlr4/org/apache/hive/hplsql/Hplsql.g4     |  40 +++++-
 .../main/java/org/apache/hive/hplsql/Conn.java  |  10 +-
 .../main/java/org/apache/hive/hplsql/Exec.java  |  97 ++++++++++++-
 .../main/java/org/apache/hive/hplsql/Query.java |  16 +++
 .../main/java/org/apache/hive/hplsql/Stmt.java  |  69 ++++++++-
 .../main/java/org/apache/hive/hplsql/Utils.java |   7 +
 .../main/java/org/apache/hive/hplsql/Var.java   |   9 +-
 .../apache/hive/hplsql/functions/Function.java  |  12 +-
 hplsql/src/main/resources/hplsql-site.xml       |  95 +++++++++++++
 .../org/apache/hive/hplsql/TestHplsqlLocal.java |  26 +---
 .../db/create_procedure_return_cursor.sql       |  53 +++++++
 .../db/create_procedure_return_cursor2.sql      |  59 ++++++++
 hplsql/src/test/queries/local/exception2.sql    |  10 --
 hplsql/src/test/queries/local/exception3.sql    |   5 -
 hplsql/src/test/queries/local/exception4.sql    |   7 -
 hplsql/src/test/queries/local/exception5.sql    |  10 --
 .../db/create_procedure_return_cursor.out.txt   | 135 ++++++++++++++++++
 .../db/create_procedure_return_cursor2.out.txt  | 139 +++++++++++++++++++
 18 files changed, 718 insertions(+), 81 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/2240dbd6/hplsql/src/main/antlr4/org/apache/hive/hplsql/Hplsql.g4
----------------------------------------------------------------------
diff --git a/hplsql/src/main/antlr4/org/apache/hive/hplsql/Hplsql.g4 b/hplsql/src/main/antlr4/org/apache/hive/hplsql/Hplsql.g4
index 852716b..70312b2 100644
--- a/hplsql/src/main/antlr4/org/apache/hive/hplsql/Hplsql.g4
+++ b/hplsql/src/main/antlr4/org/apache/hive/hplsql/Hplsql.g4
@@ -33,6 +33,8 @@ single_block_stmt :                                      // Single BEGIN END blo
 
 stmt : 
        assignment_stmt
+     | allocate_cursor_stmt
+     | associate_locator_stmt
      | break_stmt
      | call_stmt
      | close_stmt
@@ -117,6 +119,14 @@ assignment_stmt_select_item :
        (ident | (T_OPEN_P ident (T_COMMA ident)* T_CLOSE_P)) T_COLON? T_EQUAL T_OPEN_P select_stmt T_CLOSE_P
      ;
      
+allocate_cursor_stmt:
+       T_ALLOCATE ident T_CURSOR T_FOR ((T_RESULT T_SET) | T_PROCEDURE) ident
+     ;
+     
+associate_locator_stmt : 
+       T_ASSOCIATE (T_RESULT T_SET)? (T_LOCATOR | T_LOCATORS) T_OPEN_P ident (T_COMMA ident)* T_CLOSE_P T_WITH T_PROCEDURE ident
+     ;       
+
 break_stmt :
        T_BREAK
      ;
@@ -151,12 +161,15 @@ declare_condition_item :    // Condition declaration
      ;
      
 declare_cursor_item :      // Cursor declaration 
-       (T_CURSOR ident | ident T_CURSOR) declare_cursor_return? (T_IS | T_AS | T_FOR) (select_stmt | expr )
+       (T_CURSOR ident | ident T_CURSOR) (cursor_with_return | cursor_without_return)? (T_IS | T_AS | T_FOR) (select_stmt | expr )
+     ;
+     
+cursor_with_return :
+       T_WITH T_RETURN T_ONLY? (T_TO (T_CALLER | T_CLIENT))?
      ;
      
-declare_cursor_return :
+cursor_without_return :
        T_WITHOUT T_RETURN
-     | T_WITH T_RETURN T_ONLY? (T_TO (T_CALLER | T_CLIENT))?
      ;
 
 declare_handler_item :     // Condition handler declaration 
@@ -238,6 +251,7 @@ dtype :                  // Data types
      | T_INT
      | T_INTEGER
      | T_NUMBER
+     | T_RESULT_SET_LOCATOR T_VARYING
      | T_SMALLINT
      | T_STRING
      | T_TIMESTAMP
@@ -261,7 +275,7 @@ dtype_default :         // Default clause in variable declaration
      ;
      
 create_function_stmt : 
-      (T_ALTER | T_CREATE (T_OR T_REPLACE)? | T_REPLACE) T_FUNCTION ident create_routine_params create_function_return (T_AS | T_IS)? single_block_stmt 
+      (T_ALTER | T_CREATE (T_OR T_REPLACE)? | T_REPLACE) T_FUNCTION ident create_routine_params? create_function_return (T_AS | T_IS)? single_block_stmt 
     ;
      
 create_function_return :
@@ -269,7 +283,7 @@ create_function_return :
      ;
 
 create_procedure_stmt : 
-      (T_ALTER | T_CREATE (T_OR T_REPLACE)? | T_REPLACE) (T_PROCEDURE | T_PROC) ident create_routine_params create_routine_options? (T_AS | T_IS)? label? single_block_stmt (ident T_SEMICOLON)? 
+      (T_ALTER | T_CREATE (T_OR T_REPLACE)? | T_REPLACE) (T_PROCEDURE | T_PROC) ident create_routine_params? create_routine_options? (T_AS | T_IS)? label? single_block_stmt (ident T_SEMICOLON)? 
     ;
 
 create_routine_params :
@@ -287,7 +301,7 @@ create_routine_options :
 create_routine_option :
        T_LANGUAGE T_SQL       
      | T_SQL T_SECURITY (T_CREATOR | T_DEFINER | T_INVOKER | T_OWNER)
-     | T_DYNAMIC T_RESULT T_SETS L_INT
+     | T_DYNAMIC? T_RESULT T_SETS L_INT
      ;
      
 drop_stmt :             // DROP statement
@@ -886,10 +900,12 @@ null_const :                              // NULL constant
 non_reserved_words :                      // Tokens that are not reserved words and can be used as identifiers
        T_ACTIVITY_COUNT
      | T_ALL 
+     | T_ALLOCATE
      | T_ALTER
      | T_AND
      | T_AS     
-     | T_ASC    
+     | T_ASC   
+     | T_ASSOCIATE     
      | T_AT
      | T_AVG
      | T_BATCHSIZE
@@ -1004,6 +1020,8 @@ non_reserved_words :                      // Tokens that are not reserved words
      | T_LIMIT  
      | T_LINES     
      | T_LOCAL     
+     | T_LOCATOR
+     | T_LOCATORS
      | T_LOGGED     
      | T_LOOP    
      | T_MAP  
@@ -1042,6 +1060,7 @@ non_reserved_words :                      // Tokens that are not reserved words
      | T_REPLACE
      | T_RESIGNAL
      | T_RESULT
+     | T_RESULT_SET_LOCATOR
      | T_RETURN       
      | T_RETURNS
      | T_REVERSE    
@@ -1092,6 +1111,7 @@ non_reserved_words :                      // Tokens that are not reserved words
      | T_VAR
      | T_VARCHAR      
      | T_VARCHAR2
+     | T_VARYING
      | T_VARIANCE
      | T_VOLATILE
      // T_WHEN reserved word         
@@ -1104,10 +1124,12 @@ non_reserved_words :                      // Tokens that are not reserved words
 
 // Lexer rules
 T_ALL             : A L L ;
+T_ALLOCATE        : A L L O C A T E ;
 T_ALTER           : A L T E R ;
 T_AND             : A N D ;
 T_AS              : A S ;
 T_ASC             : A S C ;
+T_ASSOCIATE       : A S S O C I A T E ; 
 T_AT              : A T ;
 T_AVG             : A V G ; 
 T_BATCHSIZE       : B A T C H S I Z E ;
@@ -1214,6 +1236,8 @@ T_LIKE            : L I K E ;
 T_LIMIT           : L I M I T ;
 T_LINES           : L I N E S ; 
 T_LOCAL           : L O C A L ;
+T_LOCATOR         : L O C A T O R ; 
+T_LOCATORS        : L O C A T O R S ; 
 T_LOGGED          : L O G G E D ; 
 T_LOOP            : L O O P ;
 T_MAP             : M A P ; 
@@ -1249,6 +1273,7 @@ T_REGEXP          : R E G E X P ;
 T_REPLACE         : R E P L A C E ; 
 T_RESIGNAL        : R E S I G N A L ;
 T_RESULT          : R E S U L T ; 
+T_RESULT_SET_LOCATOR : R E S U L T '_' S E T '_' L O C A T O R ;
 T_RETURN          : R E T U R N ;
 T_RETURNS         : R E T U R N S ;
 T_REVERSE         : R E V E R S E ;
@@ -1296,6 +1321,7 @@ T_VALUES          : V A L U E S ;
 T_VAR             : V A R ;
 T_VARCHAR         : V A R C H A R ;
 T_VARCHAR2        : V A R C H A R '2' ;
+T_VARYING         : V A R Y I N G ;
 T_VOLATILE        : V O L A T I L E ;
 T_WHEN            : W H E N ;
 T_WHERE           : W H E R E ;

http://git-wip-us.apache.org/repos/asf/hive/blob/2240dbd6/hplsql/src/main/java/org/apache/hive/hplsql/Conn.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Conn.java b/hplsql/src/main/java/org/apache/hive/hplsql/Conn.java
index 828fbc3..ac4b521 100644
--- a/hplsql/src/main/java/org/apache/hive/hplsql/Conn.java
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Conn.java
@@ -41,10 +41,12 @@ public class Conn {
   Exec exec;
   Timer timer = new Timer();
   boolean trace = false;  
+  boolean info = false;
   
   Conn(Exec e) {
     exec = e;  
     trace = exec.getTrace();
+    info = exec.getInfo();
   }
   
   /**
@@ -59,8 +61,8 @@ public class Conn {
       ResultSet rs = stmt.executeQuery(query.sql);
       timer.stop();
       query.set(conn, stmt, rs);      
-      if (trace) {
-        exec.trace(null, "Query executed successfully (" + timer.format() + ")");
+      if (info) {
+        exec.info(null, "Query executed successfully (" + timer.format() + ")");
       }      
     } catch (Exception e) {
       query.setError(e);
@@ -169,8 +171,8 @@ public class Conn {
     timer.start();
     Connection conn = DriverManager.getConnection(url, usr, pwd);
     timer.stop();
-    if (trace) {
-      exec.trace(null, "Open connection: " + url + " (" + timer.format() + ")");
+    if (info) {
+      exec.info(null, "Open connection: " + url + " (" + timer.format() + ")");
     }
     return conn;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/2240dbd6/hplsql/src/main/java/org/apache/hive/hplsql/Exec.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Exec.java b/hplsql/src/main/java/org/apache/hive/hplsql/Exec.java
index 40fdc82..b35344f 100644
--- a/hplsql/src/main/java/org/apache/hive/hplsql/Exec.java
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Exec.java
@@ -39,6 +39,7 @@ import org.antlr.v4.runtime.Token;
 import org.antlr.v4.runtime.misc.NotNull;
 import org.antlr.v4.runtime.tree.ParseTree;
 import org.apache.commons.io.FileUtils;
+import org.apache.hive.hplsql.Var.Type;
 import org.apache.hive.hplsql.functions.*;
 
 /**
@@ -63,6 +64,7 @@ public class Exec extends HplsqlBaseVisitor<Integer> {
   
   Stack<Var> stack = new Stack<Var>();
   Stack<String> labels = new Stack<String>();
+  Stack<String> callStack = new Stack<String>();
   
   Stack<Signal> signals = new Stack<Signal>();
   Signal currentSignal;
@@ -72,9 +74,10 @@ public class Exec extends HplsqlBaseVisitor<Integer> {
   HashMap<String, String> managedTables = new HashMap<String, String>();
   HashMap<String, String> objectMap = new HashMap<String, String>(); 
   HashMap<String, String> objectConnMap = new HashMap<String, String>();
+  HashMap<String, ArrayList<Var>> returnCursors = new HashMap<String, ArrayList<Var>>();
   
   public ArrayList<String> stmtConnList = new ArrayList<String>();
-  
+      
   Arguments arguments = new Arguments();
   public Conf conf;
   Expression expr;
@@ -183,6 +186,32 @@ public class Exec extends HplsqlBaseVisitor<Integer> {
   }
   
   /**
+   * Add a return cursor visible to procedure callers and clients
+   */
+  public void addReturnCursor(Var var) {
+    String routine = callStackPeek();
+    ArrayList<Var> cursors = returnCursors.get(routine);
+    if (cursors == null) {
+      cursors = new ArrayList<Var>();
+      returnCursors.put(routine, cursors);
+    }
+    cursors.add(var);
+  }
+  
+  /**
+   * Get the return cursor defined in the specified procedure
+   */
+  public Var consumeReturnCursor(String routine) {
+    ArrayList<Var> cursors = returnCursors.get(routine.toUpperCase());
+    if (cursors == null) {
+      return null;
+    }
+    Var var = cursors.get(0);
+    cursors.remove(0);
+    return var;
+  }
+  
+  /**
    * Push a value to the stack
    */
   public void stackPush(Var var) {
@@ -224,6 +253,33 @@ public class Exec extends HplsqlBaseVisitor<Integer> {
     return null;
   }    
   
+  /**
+   * Push a value to the call stack
+   */
+  public void callStackPush(String val) {
+    exec.callStack.push(val.toUpperCase());  
+  }
+  
+  /**
+   * Select a value from the call stack, but not remove
+   */
+  public String callStackPeek() {
+    if (!exec.callStack.isEmpty()) {
+      return exec.callStack.peek();
+    }
+    return null;
+  }
+  
+  /**
+   * Pop a value from the call stack
+   */
+  public String callStackPop() {
+    if (!exec.callStack.isEmpty()) {
+      return exec.callStack.pop();
+    }
+    return null;
+  }  
+  
   /** 
    * Find an existing variable by name 
    */
@@ -250,6 +306,17 @@ public class Exec extends HplsqlBaseVisitor<Integer> {
   }
   
   /**
+   * Find a cursor variable by name
+   */
+  public Var findCursor(String name) {
+    Var cursor = exec.findVariable(name);
+    if (cursor != null && cursor.type == Type.CURSOR) {
+      return cursor;
+    }    
+    return null;
+  }
+  
+  /**
    * Enter a new scope
    */
   public void enterScope(Scope.Type type) {
@@ -286,10 +353,12 @@ public class Exec extends HplsqlBaseVisitor<Integer> {
   }
   
   public void signal(Signal.Type type, String value) {
+    setSqlCode(-1);
     signal(type, value, null);   
   }
   
   public void signal(Signal.Type type) {
+    setSqlCode(-1);
     signal(type, null, null);   
   }
   
@@ -480,20 +549,20 @@ public class Exec extends HplsqlBaseVisitor<Integer> {
       Entry<String,String> item = (Entry<String,String>)i.next();
       String key = (String)item.getKey();
       String value = (String)item.getValue();
-      if (key == null || value == null) {
+      if (key == null || value == null || !key.startsWith("hplsql.")) {
         continue;
       }
       else if (key.compareToIgnoreCase(Conf.CONN_DEFAULT) == 0) {
         exec.conf.defaultConnection = value;
       }
       else if (key.startsWith("hplsql.conn.init.")) {
-        exec.conn.addConnectionInit(key.substring(16), value);        
+        exec.conn.addConnectionInit(key.substring(17), value);        
       }
       else if (key.startsWith(Conf.CONN_CONVERT)) {
-        exec.conf.setConnectionConvert(key.substring(19), value);        
+        exec.conf.setConnectionConvert(key.substring(20), value);        
       }
       else if (key.startsWith("hplsql.conn.")) {
-        exec.conn.addConnection(key.substring(11), value);
+        exec.conn.addConnection(key.substring(12), value);
       }
       else if (key.startsWith("hplsql.")) {
         exec.conf.setOption(key, value);
@@ -940,7 +1009,7 @@ public class Exec extends HplsqlBaseVisitor<Integer> {
    */
   @Override
   public Integer visitDeclare_var_item(HplsqlParser.Declare_var_itemContext ctx) { 
-    String type = ctx.dtype().getText();
+    String type = getFormattedText(ctx.dtype());
     String len = null;
     String scale = null;
     Var default_ = null;
@@ -969,6 +1038,22 @@ public class Exec extends HplsqlBaseVisitor<Integer> {
 	  }	
 	  return 0;
   }
+  
+  /**
+   * ALLOCATE CURSOR statement
+   */
+  @Override 
+  public Integer visitAllocate_cursor_stmt(HplsqlParser.Allocate_cursor_stmtContext ctx) { 
+    return exec.stmt.allocateCursor(ctx); 
+  }
+
+  /**
+   * ASSOCIATE LOCATOR statement
+   */
+  @Override 
+  public Integer visitAssociate_locator_stmt(HplsqlParser.Associate_locator_stmtContext ctx) { 
+    return exec.stmt.associateLocator(ctx); 
+  }
 
   /**
    * DECLARE cursor statement

http://git-wip-us.apache.org/repos/asf/hive/blob/2240dbd6/hplsql/src/main/java/org/apache/hive/hplsql/Query.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Query.java b/hplsql/src/main/java/org/apache/hive/hplsql/Query.java
index 23d963f..eaaaa67 100644
--- a/hplsql/src/main/java/org/apache/hive/hplsql/Query.java
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Query.java
@@ -35,6 +35,8 @@ public class Query {
   ResultSet rs;
   Exception exception;
   
+  boolean withReturn = false;
+  
   Query() {
   }
   
@@ -103,6 +105,13 @@ public class Query {
   }
   
   /**
+   * Set whether the cursor is returned to the caller
+   */
+  public void setWithReturn(boolean withReturn) {
+    this.withReturn = withReturn;
+  }
+  
+  /**
    * Set an execution error
    */
   public void setError(Exception e) {
@@ -133,6 +142,13 @@ public class Query {
   }
   
   /**
+   * Check if the cursor defined as a return cursor to client
+   */
+  public boolean getWithReturn() {
+    return withReturn;
+  }
+  
+  /**
    * Return error information
    */
   public boolean error() {

http://git-wip-us.apache.org/repos/asf/hive/blob/2240dbd6/hplsql/src/main/java/org/apache/hive/hplsql/Stmt.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Stmt.java b/hplsql/src/main/java/org/apache/hive/hplsql/Stmt.java
index acc4907..bfb76cd 100644
--- a/hplsql/src/main/java/org/apache/hive/hplsql/Stmt.java
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Stmt.java
@@ -48,6 +48,58 @@ public class Stmt {
   }
   
   /**
+   * ALLOCATE CURSOR statement
+   */
+  public Integer allocateCursor(HplsqlParser.Allocate_cursor_stmtContext ctx) { 
+    trace(ctx, "ALLOCATE CURSOR");
+    String name = ctx.ident(0).getText();
+    Var cur = null;
+    if (ctx.T_PROCEDURE() != null) {
+      cur = exec.consumeReturnCursor(ctx.ident(1).getText());
+    }
+    else if (ctx.T_RESULT() != null) {
+      cur = exec.findVariable(ctx.ident(1).getText());
+      if (cur != null && cur.type != Type.RS_LOCATOR) {
+        cur = null;
+      }
+    }
+    if (cur == null) {
+      trace(ctx, "Cursor for procedure not found: " + name);
+      exec.signal(Signal.Type.SQLEXCEPTION);
+      return -1;
+    }
+    exec.addVariable(new Var(name, Type.CURSOR, cur.value)); 
+    return 0; 
+  }
+  
+  /**
+   * ASSOCIATE LOCATOR statement
+   */
+  public Integer associateLocator(HplsqlParser.Associate_locator_stmtContext ctx) { 
+    trace(ctx, "ASSOCIATE LOCATOR");
+    int cnt = ctx.ident().size();
+    if (cnt < 2) {
+      return -1;
+    }
+    String procedure = ctx.ident(cnt - 1).getText();
+    for (int i = 0; i < cnt - 1; i++) {
+      Var cur = exec.consumeReturnCursor(procedure);
+      if (cur != null) {
+        String name = ctx.ident(i).getText(); 
+        Var loc = exec.findVariable(name);
+        if (loc == null) {
+          loc = new Var(name, Type.RS_LOCATOR, cur.value);
+          exec.addVariable(loc);
+        }
+        else {
+          loc.setValue(cur.value);
+        }
+      }      
+    }
+    return 0; 
+  }
+  
+  /**
    * DECLARE cursor statement
    */
   public Integer declareCursor(HplsqlParser.Declare_cursor_itemContext ctx) { 
@@ -62,7 +114,11 @@ public class Stmt {
     else if (ctx.select_stmt() != null) {
       query.setSelectCtx(ctx.select_stmt());
     }
-    exec.addVariable(new Var(name, Type.CURSOR, query));
+    if (ctx.cursor_with_return() != null) {
+      query.setWithReturn(true);
+    }
+    Var var = new Var(name, Type.CURSOR, query);
+    exec.addVariable(var);
     return 0; 
   }
   
@@ -262,6 +318,9 @@ public class Stmt {
       else if (!exec.getOffline()) {
         exec.setSqlCode(0);
       }
+      if (query.getWithReturn()) {
+        exec.addReturnCursor(var);
+      }
     }
     else {
       trace(ctx, "Cursor not found: " + cursor);
@@ -278,8 +337,8 @@ public class Stmt {
   public Integer fetch(HplsqlParser.Fetch_stmtContext ctx) { 
     trace(ctx, "FETCH");
     String name = ctx.L_ID(0).toString();
-    Var cursor = exec.findVariable(name);
-    if (cursor == null || cursor.type != Type.CURSOR) {
+    Var cursor = exec.findCursor(name);
+    if (cursor == null) {
       trace(ctx, "Cursor not found: " + name);
       exec.setSqlCode(-1);
       exec.signal(Signal.Type.SQLEXCEPTION);
@@ -319,9 +378,11 @@ public class Stmt {
         }
         else {
           exec.setSqlCode(100);
-          exec.signal(Signal.Type.NOTFOUND);
         }
       }
+      else {
+        exec.setSqlCode(-1);
+      }
     } 
     catch (SQLException e) {
       exec.setSqlCode(e);

http://git-wip-us.apache.org/repos/asf/hive/blob/2240dbd6/hplsql/src/main/java/org/apache/hive/hplsql/Utils.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Utils.java b/hplsql/src/main/java/org/apache/hive/hplsql/Utils.java
index da0d878..1815deb 100644
--- a/hplsql/src/main/java/org/apache/hive/hplsql/Utils.java
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Utils.java
@@ -286,4 +286,11 @@ public class Utils {
     float bytesPerSec = ((float)bytes)/msElapsed*1000;
     return Utils.formatSizeInBytes((long)bytesPerSec, "/sec");
   }
+  
+  /**
+   * Note. This stub is to resolve name conflict with ANTLR generated source using org.antlr.v4.runtime.misc.Utils.join
+   */
+  static <T> String join(T[] array, String separator) {
+    return org.antlr.v4.runtime.misc.Utils.join(array, separator);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/2240dbd6/hplsql/src/main/java/org/apache/hive/hplsql/Var.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Var.java b/hplsql/src/main/java/org/apache/hive/hplsql/Var.java
index 0a4ead2..87b42f9 100644
--- a/hplsql/src/main/java/org/apache/hive/hplsql/Var.java
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Var.java
@@ -32,7 +32,7 @@ import java.sql.Timestamp;
 public class Var {
 
 	// Data types
-	public enum Type {BOOL, CURSOR, DATE, DEC, FILE, IDENT, BIGINT, INTERVAL, STRING, STRINGLIST, TIMESTAMP, NULL};
+	public enum Type {BOOL, CURSOR, DATE, DEC, FILE, IDENT, BIGINT, INTERVAL, RS_LOCATOR, STRING, STRINGLIST, TIMESTAMP, NULL};
 	public static Var Empty = new Var();
 	public static Var Null = new Var(Type.NULL);
 	
@@ -194,6 +194,10 @@ public class Var {
     return this;
   }
 	
+	public void setValue(Object value) {
+    this.value = value;
+  }
+	
 	/**
    * Set the new value from a result set
    */
@@ -244,6 +248,9 @@ public class Var {
     else if (type.equalsIgnoreCase("UTL_FILE.FILE_TYPE")) {
       return Type.FILE;
     }
+    else if (type.toUpperCase().startsWith("RESULT_SET_LOCATOR")) {
+      return Type.RS_LOCATOR;
+    }
     return Type.NULL;
   }
   

http://git-wip-us.apache.org/repos/asf/hive/blob/2240dbd6/hplsql/src/main/java/org/apache/hive/hplsql/functions/Function.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/functions/Function.java b/hplsql/src/main/java/org/apache/hive/hplsql/functions/Function.java
index 9895b5e..394598b 100644
--- a/hplsql/src/main/java/org/apache/hive/hplsql/functions/Function.java
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/functions/Function.java
@@ -188,8 +188,12 @@ public class Function {
       return false;
     }    
     exec.enterScope(Scope.Type.ROUTINE);
-    setCallParameters(procCtx.create_routine_params());
+    exec.callStackPush(name);
+    if (procCtx.create_routine_params() != null) {
+      setCallParameters(procCtx.create_routine_params());
+    }
     visit(procCtx.single_block_stmt());
+    exec.callStackPop();
     exec.leaveScope();       
     return true;
   }
@@ -208,8 +212,12 @@ public class Function {
     }    
     HashMap<String, Var> out = new HashMap<String, Var>();
     exec.enterScope(Scope.Type.ROUTINE);
-    setCallParameters(ctx, procCtx.create_routine_params(), out);
+    exec.callStackPush(name);
+    if (procCtx.create_routine_params() != null) {
+      setCallParameters(ctx, procCtx.create_routine_params(), out);
+    }
     visit(procCtx.single_block_stmt());
+    exec.callStackPop();
     exec.leaveScope();       
     for (Map.Entry<String, Var> i : out.entrySet()) {      // Set OUT parameters
       exec.setVariable(i.getKey(), i.getValue());

http://git-wip-us.apache.org/repos/asf/hive/blob/2240dbd6/hplsql/src/main/resources/hplsql-site.xml
----------------------------------------------------------------------
diff --git a/hplsql/src/main/resources/hplsql-site.xml b/hplsql/src/main/resources/hplsql-site.xml
new file mode 100644
index 0000000..1a3202a
--- /dev/null
+++ b/hplsql/src/main/resources/hplsql-site.xml
@@ -0,0 +1,95 @@
+<configuration>
+<property>
+  <name>hplsql.conn.default</name>
+  <value>hiveconn</value>
+  <description>The default connection profile</description>
+</property>
+<property>
+  <name>hplsql.conn.hiveconn</name>
+  <value>org.apache.hive.jdbc.HiveDriver;jdbc:hive2://</value>
+  <description>HiveServer2 JDBC connection (embedded mode)</description>
+</property>
+<property>
+  <name>hplsql.conn.init.hiveconn</name>
+  <value>
+     set mapred.job.queue.name=default;
+     set hive.execution.engine=mr; 
+     use default;
+  </value>
+  <description>Statements for execute after connection to the database</description>
+</property>
+<property>
+  <name>hplsql.conn.convert.hiveconn</name>
+  <value>true</value>
+  <description>Convert SQL statements before execution</description>
+</property>
+<property>
+  <name>hplsql.conn.hive1conn</name>
+  <value>org.apache.hadoop.hive.jdbc.HiveDriver;jdbc:hive://</value>
+  <description>Hive embedded JDBC (not requiring HiveServer)</description>
+</property>
+<property>
+  <name>hplsql.conn.hive2conn</name>
+  <value>org.apache.hive.jdbc.HiveDriver;jdbc:hive2://localhost:10000;hive;hive</value>
+  <description>HiveServer2 JDBC connection</description>
+</property>
+<property>
+  <name>hplsql.conn.init.hive2conn</name>
+  <value>
+     set mapred.job.queue.name=default;
+     set hive.execution.engine=mr; 
+     use default;
+  </value>
+  <description>Statements for execute after connection to the database</description>
+</property>
+<property>
+  <name>hplsql.conn.convert.hive2conn</name>
+  <value>true</value>
+  <description>Convert SQL statements before execution</description>
+</property>
+<property>
+  <name>hplsql.conn.db2conn</name>
+  <value>com.ibm.db2.jcc.DB2Driver;jdbc:db2://localhost:50001/dbname;user;password</value>
+  <description>IBM DB2 connection</description>
+</property>
+<property>
+  <name>hplsql.conn.tdconn</name>
+  <value>com.teradata.jdbc.TeraDriver;jdbc:teradata://localhost/database=dbname,logmech=ldap;user;password</value>
+  <description>Teradata connection</description>
+</property>
+<property>
+  <name>hplsql.conn.mysqlconn</name>
+  <value>com.mysql.jdbc.Driver;jdbc:mysql://localhost/test;user;password</value>
+  <description>MySQL connection</description>
+</property>
+<property>
+  <name>hplsql.dual.table</name>
+  <value>default.dual</value>
+  <description>Single row, single column table for internal operations</description>
+</property>
+<property>
+  <name>hplsql.insert.values</name>
+  <value>native</value>
+  <description>How to execute INSERT VALUES statement: native (default) and select</description>
+</property>
+<property>
+  <name>hplsql.onerror</name>
+  <value>exception</value>
+  <description>Error handling behavior: exception (default), seterror and stop</description>
+</property>
+<property>
+  <name>hplsql.temp.tables</name>
+  <value>native</value>
+  <description>Temporary tables: native (default) and managed</description>
+</property>
+<property>
+  <name>hplsql.temp.tables.schema</name>
+  <value></value>
+  <description>Schema for managed temporary tables</description>
+</property>
+<property>
+  <name>hplsql.temp.tables.location</name>
+  <value>/tmp/plhql</value>
+  <description>LOcation for managed temporary tables in HDFS</description>
+</property>
+</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/2240dbd6/hplsql/src/test/java/org/apache/hive/hplsql/TestHplsqlLocal.java
----------------------------------------------------------------------
diff --git a/hplsql/src/test/java/org/apache/hive/hplsql/TestHplsqlLocal.java b/hplsql/src/test/java/org/apache/hive/hplsql/TestHplsqlLocal.java
index ee2be66..5ec91d9 100644
--- a/hplsql/src/test/java/org/apache/hive/hplsql/TestHplsqlLocal.java
+++ b/hplsql/src/test/java/org/apache/hive/hplsql/TestHplsqlLocal.java
@@ -134,26 +134,6 @@ public class TestHplsqlLocal {
   }
 
   @Test
-  public void testException2() throws Exception {
-    run("exception2");
-  }
-
-  @Test
-  public void testException3() throws Exception {
-    run("exception2");
-  }
-
-  @Test
-  public void testException4() throws Exception {
-    run("exception2");
-  }
-
-  @Test
-  public void testException5() throws Exception {
-    run("exception2");
-  }
-
-  @Test
   public void testExit() throws Exception {
     run("exit");
   }
@@ -300,11 +280,7 @@ public class TestHplsqlLocal {
     System.setOut(new PrintStream(out));
     Exec exec = new Exec();
     String[] args = { "-f", "src/test/queries/local/" + testFile + ".sql", "-trace" };
-    exec.init(args);
-    Var result = exec.run();
-    if (result != null) {
-      System.out.println(result.toString());
-    }
+    exec.run(args);
     String s = getTestOutput(out.toString()).trim();
     FileUtils.writeStringToFile(new java.io.File("target/tmp/log/" + testFile + ".out.txt"), s);
     String t = FileUtils.readFileToString(new java.io.File("src/test/results/local/" + testFile + ".out.txt"), "utf-8").trim();

http://git-wip-us.apache.org/repos/asf/hive/blob/2240dbd6/hplsql/src/test/queries/db/create_procedure_return_cursor.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/db/create_procedure_return_cursor.sql b/hplsql/src/test/queries/db/create_procedure_return_cursor.sql
new file mode 100644
index 0000000..d954863
--- /dev/null
+++ b/hplsql/src/test/queries/db/create_procedure_return_cursor.sql
@@ -0,0 +1,53 @@
+CREATE PROCEDURE spResultSet1 
+  DYNAMIC RESULT SETS 1
+BEGIN
+  DECLARE cur1 CURSOR WITH RETURN FOR
+    SELECT 'A', 'A1' FROM src LIMIT 3;
+  OPEN cur1;
+END;  
+  
+CREATE PROCEDURE spResultSet2
+  DYNAMIC RESULT SETS 2
+BEGIN
+  DECLARE cur1 CURSOR WITH RETURN FOR
+    SELECT 'B', 'B1' FROM src LIMIT 5;
+  DECLARE cur2 CURSOR WITH RETURN FOR
+    SELECT 'C', 'C1' FROM src LIMIT 7;
+  OPEN cur1;
+  OPEN cur2;
+END;
+
+DECLARE v1 VARCHAR(10);
+DECLARE v2 VARCHAR(10);
+
+CALL spResultSet1;
+ALLOCATE c1 CURSOR FOR PROCEDURE spResultSet1;
+
+FETCH c1 INTO v1, v2;
+WHILE (SQLCODE = 0)
+DO
+  PRINT v1 || ' - ' || v2;
+  FETCH c1 INTO v1, v2;
+END WHILE;
+CLOSE c1;
+
+CALL spResultSet2;
+ALLOCATE c2 CURSOR FOR PROCEDURE spResultSet2;
+
+FETCH c2 INTO v1, v2;
+WHILE (SQLCODE = 0)
+DO
+  PRINT v1 || ' - ' || v2;
+  FETCH c2 INTO v1, v2;
+END WHILE;
+CLOSE c2;
+
+ALLOCATE c3 CURSOR FOR PROCEDURE spResultSet2;
+
+FETCH c3 INTO v1, v2;
+WHILE (SQLCODE = 0)
+DO
+  PRINT v1 || ' - ' || v2;
+  FETCH c3 INTO v1, v2;
+END WHILE;
+CLOSE c3;

http://git-wip-us.apache.org/repos/asf/hive/blob/2240dbd6/hplsql/src/test/queries/db/create_procedure_return_cursor2.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/db/create_procedure_return_cursor2.sql b/hplsql/src/test/queries/db/create_procedure_return_cursor2.sql
new file mode 100644
index 0000000..a9a1ffe
--- /dev/null
+++ b/hplsql/src/test/queries/db/create_procedure_return_cursor2.sql
@@ -0,0 +1,59 @@
+CREATE PROCEDURE spResultSet1 
+  DYNAMIC RESULT SETS 1
+BEGIN
+  DECLARE cur1 CURSOR WITH RETURN FOR
+    SELECT 'A', 'A1' FROM src LIMIT 3;
+  OPEN cur1;
+END;  
+  
+CREATE PROCEDURE spResultSet2
+  DYNAMIC RESULT SETS 2
+BEGIN
+  DECLARE cur1 CURSOR WITH RETURN FOR
+    SELECT 'B', 'B1' FROM src LIMIT 5;
+  DECLARE cur2 CURSOR WITH RETURN FOR
+    SELECT 'C', 'C1' FROM src LIMIT 7;
+  OPEN cur1;
+  OPEN cur2;
+END;
+
+DECLARE v1 VARCHAR(10);
+DECLARE v2 VARCHAR(10);
+DECLARE loc1 RESULT_SET_LOCATOR VARYING;
+DECLARE loc2 RESULT_SET_LOCATOR VARYING;
+
+CALL spResultSet1;
+
+ASSOCIATE RESULT SET LOCATOR (loc1) WITH PROCEDURE spResultSet1;
+ALLOCATE c1 CURSOR FOR RESULT SET loc1;
+
+FETCH c1 INTO v1, v2;
+WHILE (SQLCODE = 0)
+DO
+  PRINT v1 || ' - ' || v2;
+  FETCH c1 INTO v1, v2;
+END WHILE;
+CLOSE c1;
+
+CALL spResultSet2;
+
+ASSOCIATE RESULT SET LOCATOR (loc1, loc2) WITH PROCEDURE spResultSet2;
+ALLOCATE c2 CURSOR FOR RESULT SET loc1;
+
+FETCH c2 INTO v1, v2;
+WHILE (SQLCODE = 0)
+DO
+  PRINT v1 || ' - ' || v2;
+  FETCH c2 INTO v1, v2;
+END WHILE;
+CLOSE c2;
+
+ALLOCATE c3 CURSOR FOR RESULT SET loc2;
+
+FETCH c3 INTO v1, v2;
+WHILE (SQLCODE = 0)
+DO
+  PRINT v1 || ' - ' || v2;
+  FETCH c3 INTO v1, v2;
+END WHILE;
+CLOSE c3;

http://git-wip-us.apache.org/repos/asf/hive/blob/2240dbd6/hplsql/src/test/queries/local/exception2.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/exception2.sql b/hplsql/src/test/queries/local/exception2.sql
deleted file mode 100644
index 3394da8..0000000
--- a/hplsql/src/test/queries/local/exception2.sql
+++ /dev/null
@@ -1,10 +0,0 @@
-DECLARE 
-  v VARCHAR(200);
-BEGIN
-  OPEN cur FOR 'SELECT c1 FROM t1';
-  FETCH cur INTO v;
-  CLOSE cur;
-EXCEPTION WHEN OTHERS THEN
-  DBMS_OUTPUT.PUT_LINE('Error');
-END 
-

http://git-wip-us.apache.org/repos/asf/hive/blob/2240dbd6/hplsql/src/test/queries/local/exception3.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/exception3.sql b/hplsql/src/test/queries/local/exception3.sql
deleted file mode 100644
index a12b853..0000000
--- a/hplsql/src/test/queries/local/exception3.sql
+++ /dev/null
@@ -1,5 +0,0 @@
-PRINT 'Correct';
-WHILE 1=1 THEN
-FETCH cur INTO v;
-PRINT 'Incorrect - unreachable code, unknown cursor name, exception must be raised';
-END WHILE;

http://git-wip-us.apache.org/repos/asf/hive/blob/2240dbd6/hplsql/src/test/queries/local/exception4.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/exception4.sql b/hplsql/src/test/queries/local/exception4.sql
deleted file mode 100644
index 38d89b5..0000000
--- a/hplsql/src/test/queries/local/exception4.sql
+++ /dev/null
@@ -1,7 +0,0 @@
-PRINT 'Correct';
-DECLARE EXIT HANDLER FOR SQLEXCEPTION
-  PRINT 'Correct - Exception raised';    
-WHILE 1=1 THEN
-FETCH cur INTO v;
-PRINT 'Incorrect - unreachable code, unknown cursor name, exception must be raised';
-END WHILE;

http://git-wip-us.apache.org/repos/asf/hive/blob/2240dbd6/hplsql/src/test/queries/local/exception5.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/exception5.sql b/hplsql/src/test/queries/local/exception5.sql
deleted file mode 100644
index 6232984..0000000
--- a/hplsql/src/test/queries/local/exception5.sql
+++ /dev/null
@@ -1,10 +0,0 @@
-DECLARE cnt INT := 0;
-PRINT 'Correct';
-DECLARE CONTINUE HANDLER FOR SQLEXCEPTION
-  PRINT 'Correct - Exception raised';    
-WHILE cnt < 10 THEN
-FETCH cur INTO v;
-PRINT cnt;
-PRINT 'Correct - exception handled';
-SET cnt = cnt + 1;
-END WHILE;

http://git-wip-us.apache.org/repos/asf/hive/blob/2240dbd6/hplsql/src/test/results/db/create_procedure_return_cursor.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/db/create_procedure_return_cursor.out.txt b/hplsql/src/test/results/db/create_procedure_return_cursor.out.txt
new file mode 100644
index 0000000..81aa6c8
--- /dev/null
+++ b/hplsql/src/test/results/db/create_procedure_return_cursor.out.txt
@@ -0,0 +1,135 @@
+Ln:1 CREATE PROCEDURE spResultSet1
+Ln:9 CREATE PROCEDURE spResultSet2
+Ln:20 DECLARE v1 VARCHAR
+Ln:21 DECLARE v2 VARCHAR
+EXEC PROCEDURE spResultSet1
+Ln:4 DECLARE CURSOR cur1
+Ln:6 OPEN
+Ln:6 cur1: SELECT 'A', 'A1' FROM src LIMIT 3
+Ln:24 ALLOCATE CURSOR
+Ln:26 FETCH
+Ln:26 COLUMN: _c0, string
+Ln:26 SET v1 = A
+Ln:26 COLUMN: _c1, string
+Ln:26 SET v2 = A1
+Ln:27 WHILE - ENTERED
+Ln:29 PRINT
+A - A1
+Ln:30 FETCH
+Ln:30 COLUMN: _c0, string
+Ln:30 SET v1 = A
+Ln:30 COLUMN: _c1, string
+Ln:30 SET v2 = A1
+Ln:29 PRINT
+A - A1
+Ln:30 FETCH
+Ln:30 COLUMN: _c0, string
+Ln:30 SET v1 = A
+Ln:30 COLUMN: _c1, string
+Ln:30 SET v2 = A1
+Ln:29 PRINT
+A - A1
+Ln:30 FETCH
+Ln:27 WHILE - LEFT
+Ln:32 CLOSE
+EXEC PROCEDURE spResultSet2
+Ln:12 DECLARE CURSOR cur1
+Ln:14 DECLARE CURSOR cur2
+Ln:16 OPEN
+Ln:16 cur1: SELECT 'B', 'B1' FROM src LIMIT 5
+Ln:17 OPEN
+Ln:17 cur2: SELECT 'C', 'C1' FROM src LIMIT 7
+Ln:35 ALLOCATE CURSOR
+Ln:37 FETCH
+Ln:37 COLUMN: _c0, string
+Ln:37 SET v1 = B
+Ln:37 COLUMN: _c1, string
+Ln:37 SET v2 = B1
+Ln:38 WHILE - ENTERED
+Ln:40 PRINT
+B - B1
+Ln:41 FETCH
+Ln:41 COLUMN: _c0, string
+Ln:41 SET v1 = B
+Ln:41 COLUMN: _c1, string
+Ln:41 SET v2 = B1
+Ln:40 PRINT
+B - B1
+Ln:41 FETCH
+Ln:41 COLUMN: _c0, string
+Ln:41 SET v1 = B
+Ln:41 COLUMN: _c1, string
+Ln:41 SET v2 = B1
+Ln:40 PRINT
+B - B1
+Ln:41 FETCH
+Ln:41 COLUMN: _c0, string
+Ln:41 SET v1 = B
+Ln:41 COLUMN: _c1, string
+Ln:41 SET v2 = B1
+Ln:40 PRINT
+B - B1
+Ln:41 FETCH
+Ln:41 COLUMN: _c0, string
+Ln:41 SET v1 = B
+Ln:41 COLUMN: _c1, string
+Ln:41 SET v2 = B1
+Ln:40 PRINT
+B - B1
+Ln:41 FETCH
+Ln:38 WHILE - LEFT
+Ln:43 CLOSE
+Ln:45 ALLOCATE CURSOR
+Ln:47 FETCH
+Ln:47 COLUMN: _c0, string
+Ln:47 SET v1 = C
+Ln:47 COLUMN: _c1, string
+Ln:47 SET v2 = C1
+Ln:48 WHILE - ENTERED
+Ln:50 PRINT
+C - C1
+Ln:51 FETCH
+Ln:51 COLUMN: _c0, string
+Ln:51 SET v1 = C
+Ln:51 COLUMN: _c1, string
+Ln:51 SET v2 = C1
+Ln:50 PRINT
+C - C1
+Ln:51 FETCH
+Ln:51 COLUMN: _c0, string
+Ln:51 SET v1 = C
+Ln:51 COLUMN: _c1, string
+Ln:51 SET v2 = C1
+Ln:50 PRINT
+C - C1
+Ln:51 FETCH
+Ln:51 COLUMN: _c0, string
+Ln:51 SET v1 = C
+Ln:51 COLUMN: _c1, string
+Ln:51 SET v2 = C1
+Ln:50 PRINT
+C - C1
+Ln:51 FETCH
+Ln:51 COLUMN: _c0, string
+Ln:51 SET v1 = C
+Ln:51 COLUMN: _c1, string
+Ln:51 SET v2 = C1
+Ln:50 PRINT
+C - C1
+Ln:51 FETCH
+Ln:51 COLUMN: _c0, string
+Ln:51 SET v1 = C
+Ln:51 COLUMN: _c1, string
+Ln:51 SET v2 = C1
+Ln:50 PRINT
+C - C1
+Ln:51 FETCH
+Ln:51 COLUMN: _c0, string
+Ln:51 SET v1 = C
+Ln:51 COLUMN: _c1, string
+Ln:51 SET v2 = C1
+Ln:50 PRINT
+C - C1
+Ln:51 FETCH
+Ln:48 WHILE - LEFT
+Ln:53 CLOSE
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/2240dbd6/hplsql/src/test/results/db/create_procedure_return_cursor2.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/db/create_procedure_return_cursor2.out.txt b/hplsql/src/test/results/db/create_procedure_return_cursor2.out.txt
new file mode 100644
index 0000000..40f2c33
--- /dev/null
+++ b/hplsql/src/test/results/db/create_procedure_return_cursor2.out.txt
@@ -0,0 +1,139 @@
+Ln:1 CREATE PROCEDURE spResultSet1
+Ln:9 CREATE PROCEDURE spResultSet2
+Ln:20 DECLARE v1 VARCHAR
+Ln:21 DECLARE v2 VARCHAR
+Ln:22 DECLARE loc1 RESULT_SET_LOCATOR VARYING
+Ln:23 DECLARE loc2 RESULT_SET_LOCATOR VARYING
+EXEC PROCEDURE spResultSet1
+Ln:4 DECLARE CURSOR cur1
+Ln:6 OPEN
+Ln:6 cur1: SELECT 'A', 'A1' FROM src LIMIT 3
+Ln:27 ASSOCIATE LOCATOR
+Ln:28 ALLOCATE CURSOR
+Ln:30 FETCH
+Ln:30 COLUMN: _c0, string
+Ln:30 SET v1 = A
+Ln:30 COLUMN: _c1, string
+Ln:30 SET v2 = A1
+Ln:31 WHILE - ENTERED
+Ln:33 PRINT
+A - A1
+Ln:34 FETCH
+Ln:34 COLUMN: _c0, string
+Ln:34 SET v1 = A
+Ln:34 COLUMN: _c1, string
+Ln:34 SET v2 = A1
+Ln:33 PRINT
+A - A1
+Ln:34 FETCH
+Ln:34 COLUMN: _c0, string
+Ln:34 SET v1 = A
+Ln:34 COLUMN: _c1, string
+Ln:34 SET v2 = A1
+Ln:33 PRINT
+A - A1
+Ln:34 FETCH
+Ln:31 WHILE - LEFT
+Ln:36 CLOSE
+EXEC PROCEDURE spResultSet2
+Ln:12 DECLARE CURSOR cur1
+Ln:14 DECLARE CURSOR cur2
+Ln:16 OPEN
+Ln:16 cur1: SELECT 'B', 'B1' FROM src LIMIT 5
+Ln:17 OPEN
+Ln:17 cur2: SELECT 'C', 'C1' FROM src LIMIT 7
+Ln:40 ASSOCIATE LOCATOR
+Ln:41 ALLOCATE CURSOR
+Ln:43 FETCH
+Ln:43 COLUMN: _c0, string
+Ln:43 SET v1 = B
+Ln:43 COLUMN: _c1, string
+Ln:43 SET v2 = B1
+Ln:44 WHILE - ENTERED
+Ln:46 PRINT
+B - B1
+Ln:47 FETCH
+Ln:47 COLUMN: _c0, string
+Ln:47 SET v1 = B
+Ln:47 COLUMN: _c1, string
+Ln:47 SET v2 = B1
+Ln:46 PRINT
+B - B1
+Ln:47 FETCH
+Ln:47 COLUMN: _c0, string
+Ln:47 SET v1 = B
+Ln:47 COLUMN: _c1, string
+Ln:47 SET v2 = B1
+Ln:46 PRINT
+B - B1
+Ln:47 FETCH
+Ln:47 COLUMN: _c0, string
+Ln:47 SET v1 = B
+Ln:47 COLUMN: _c1, string
+Ln:47 SET v2 = B1
+Ln:46 PRINT
+B - B1
+Ln:47 FETCH
+Ln:47 COLUMN: _c0, string
+Ln:47 SET v1 = B
+Ln:47 COLUMN: _c1, string
+Ln:47 SET v2 = B1
+Ln:46 PRINT
+B - B1
+Ln:47 FETCH
+Ln:44 WHILE - LEFT
+Ln:49 CLOSE
+Ln:51 ALLOCATE CURSOR
+Ln:53 FETCH
+Ln:53 COLUMN: _c0, string
+Ln:53 SET v1 = C
+Ln:53 COLUMN: _c1, string
+Ln:53 SET v2 = C1
+Ln:54 WHILE - ENTERED
+Ln:56 PRINT
+C - C1
+Ln:57 FETCH
+Ln:57 COLUMN: _c0, string
+Ln:57 SET v1 = C
+Ln:57 COLUMN: _c1, string
+Ln:57 SET v2 = C1
+Ln:56 PRINT
+C - C1
+Ln:57 FETCH
+Ln:57 COLUMN: _c0, string
+Ln:57 SET v1 = C
+Ln:57 COLUMN: _c1, string
+Ln:57 SET v2 = C1
+Ln:56 PRINT
+C - C1
+Ln:57 FETCH
+Ln:57 COLUMN: _c0, string
+Ln:57 SET v1 = C
+Ln:57 COLUMN: _c1, string
+Ln:57 SET v2 = C1
+Ln:56 PRINT
+C - C1
+Ln:57 FETCH
+Ln:57 COLUMN: _c0, string
+Ln:57 SET v1 = C
+Ln:57 COLUMN: _c1, string
+Ln:57 SET v2 = C1
+Ln:56 PRINT
+C - C1
+Ln:57 FETCH
+Ln:57 COLUMN: _c0, string
+Ln:57 SET v1 = C
+Ln:57 COLUMN: _c1, string
+Ln:57 SET v2 = C1
+Ln:56 PRINT
+C - C1
+Ln:57 FETCH
+Ln:57 COLUMN: _c0, string
+Ln:57 SET v1 = C
+Ln:57 COLUMN: _c1, string
+Ln:57 SET v2 = C1
+Ln:56 PRINT
+C - C1
+Ln:57 FETCH
+Ln:54 WHILE - LEFT
+Ln:59 CLOSE
\ No newline at end of file


[39/43] hive git commit: HIVE-11330: Add early termination for recursion in StatsRulesProcFactory.evaluateExpression (Prasanth Jayachandran, reviewed by Hari Subramaniyan)

Posted by xu...@apache.org.
HIVE-11330: Add early termination for recursion in StatsRulesProcFactory.evaluateExpression (Prasanth Jayachandran, reviewed by Hari Subramaniyan)


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

Branch: refs/heads/spark
Commit: 4ee17e53c194501d8eaca07c3039379b56d86d26
Parents: f312d17
Author: Hari Subramaniyan <ha...@apache.org>
Authored: Wed Jul 29 14:57:31 2015 -0700
Committer: Hari Subramaniyan <ha...@apache.org>
Committed: Wed Jul 29 14:57:31 2015 -0700

----------------------------------------------------------------------
 .../hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/4ee17e53/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java
index 376d42c..1663b88 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java
@@ -301,6 +301,9 @@ public class StatsRulesProcFactory {
       long newNumRows = 0;
       Statistics andStats = null;
 
+      if (stats.getNumRows() <= 1 || stats.getDataSize() <= 0)
+        return 1;
+
       if (pred instanceof ExprNodeGenericFuncDesc) {
         ExprNodeGenericFuncDesc genFunc = (ExprNodeGenericFuncDesc) pred;
         GenericUDF udf = genFunc.getGenericUDF();


[25/43] hive git commit: HIVE-11288: Avro SerDe InstanceCache returns incorrect schema (Greg via Xuefu)

Posted by xu...@apache.org.
HIVE-11288: Avro SerDe InstanceCache returns incorrect schema (Greg via Xuefu)


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

Branch: refs/heads/spark
Commit: d2ee3543474b15d04bb3be3cdf0ad987338418a1
Parents: 65396f0
Author: xzhang <xz...@xzdt>
Authored: Tue Jul 28 11:39:37 2015 -0700
Committer: xzhang <xz...@xzdt>
Committed: Tue Jul 28 11:40:38 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hive/serde2/avro/InstanceCache.java  |  9 +++--
 .../hive/serde2/avro/TestInstanceCache.java     | 40 +++++++++++++-------
 2 files changed, 32 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/d2ee3543/serde/src/java/org/apache/hadoop/hive/serde2/avro/InstanceCache.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/avro/InstanceCache.java b/serde/src/java/org/apache/hadoop/hive/serde2/avro/InstanceCache.java
index 0b10d99..c7289cb 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/avro/InstanceCache.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/avro/InstanceCache.java
@@ -21,6 +21,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
 import java.util.HashMap;
+import java.util.Map;
 import java.util.Set;
 
 /**
@@ -33,7 +34,7 @@ import java.util.Set;
  */
 public abstract class InstanceCache<SeedObject, Instance> {
   private static final Log LOG = LogFactory.getLog(InstanceCache.class);
-  HashMap<Integer, Instance> cache = new HashMap<Integer, Instance>();
+  Map<SeedObject, Instance> cache = new HashMap<SeedObject, Instance>();
   
   public InstanceCache() {}
 
@@ -53,15 +54,15 @@ public abstract class InstanceCache<SeedObject, Instance> {
       Set<SeedObject> seenSchemas) throws AvroSerdeException {
     if(LOG.isDebugEnabled()) LOG.debug("Checking for hv: " + hv.toString());
 
-    if(cache.containsKey(hv.hashCode())) {
+    if(cache.containsKey(hv)) {
       if(LOG.isDebugEnabled()) LOG.debug("Returning cache result.");
-      return cache.get(hv.hashCode());
+      return cache.get(hv);
     }
 
     if(LOG.isDebugEnabled()) LOG.debug("Creating new instance and storing in cache");
 
     Instance instance = makeInstance(hv, seenSchemas);
-    cache.put(hv.hashCode(), instance);
+    cache.put(hv, instance);
     return instance;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/d2ee3543/serde/src/test/org/apache/hadoop/hive/serde2/avro/TestInstanceCache.java
----------------------------------------------------------------------
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/avro/TestInstanceCache.java b/serde/src/test/org/apache/hadoop/hive/serde2/avro/TestInstanceCache.java
index 814e8e4..cb7c6ed 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/avro/TestInstanceCache.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/avro/TestInstanceCache.java
@@ -24,9 +24,23 @@ import static org.junit.Assert.assertSame;
 
 public class TestInstanceCache {
   private static class Foo {
+
+    private int value = 42;
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+
+      Foo foo = (Foo) o;
+
+      return value == foo.value;
+
+    }
+
     @Override
     public int hashCode() {
-      return 42;
+      return value;
     }
   }
 
@@ -41,12 +55,12 @@ public class TestInstanceCache {
   @Test
   public void instanceCachesOnlyCreateOneInstance() throws AvroSerdeException {
     InstanceCache<Foo, Wrapper<Foo>> ic = new InstanceCache<Foo, Wrapper<Foo>>() {
-                                           @Override
-                                           protected Wrapper makeInstance(Foo hv,
-                                               Set<Foo> seenSchemas) {
-                                             return new Wrapper(hv);
-                                           }
-                                          };
+      @Override
+      protected Wrapper makeInstance(Foo hv,
+                                     Set<Foo> seenSchemas) {
+        return new Wrapper(hv);
+      }
+    };
     Foo f1 = new Foo();
 
     Wrapper fc = ic.retrieve(f1, null);
@@ -62,12 +76,12 @@ public class TestInstanceCache {
   @Test
   public void instanceCacheReturnsCorrectInstances() throws AvroSerdeException {
     InstanceCache<String, Wrapper<String>> ic = new InstanceCache<String, Wrapper<String>>() {
-                                   @Override
-                                   protected Wrapper<String> makeInstance(
-                                       String hv, Set<String> seenSchemas) {
-                                     return new Wrapper<String>(hv);
-                                   }
-                                 };
+      @Override
+      protected Wrapper<String> makeInstance(
+          String hv, Set<String> seenSchemas) {
+        return new Wrapper<String>(hv);
+      }
+    };
 
     Wrapper<String> one = ic.retrieve("one", null);
     Wrapper<String> two = ic.retrieve("two", null);


[31/43] hive git commit: HIVE-11209. Clean up dependencies in HiveDecimalWritable. (omalley reviewed by prasanthj)

Posted by xu...@apache.org.
HIVE-11209. Clean up dependencies in HiveDecimalWritable. (omalley reviewed by
prasanthj)


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

Branch: refs/heads/spark
Commit: 29651cd370120ace1d514b8aab4880936228d45e
Parents: c178a6e
Author: Owen O'Malley <om...@apache.org>
Authored: Tue Jul 28 12:53:00 2015 -0700
Committer: Owen O'Malley <om...@apache.org>
Committed: Tue Jul 28 12:53:00 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hive/common/type/HiveDecimal.java    |  6 +++
 .../hive/ql/io/orc/TreeReaderFactory.java       | 12 ++---
 .../hive/ql/udf/generic/GenericUDFBridge.java   |  8 +--
 .../apache/hadoop/hive/serde2/SerDeUtils.java   | 14 +++++-
 .../hive/serde2/io/HiveDecimalWritable.java     | 51 ++++++--------------
 .../hive/serde2/lazy/LazyHiveDecimal.java       |  3 +-
 .../lazy/fast/LazySimpleDeserializeRead.java    | 16 ++----
 .../lazybinary/LazyBinaryHiveDecimal.java       |  2 +-
 .../hive/serde2/lazybinary/LazyBinarySerDe.java | 26 +++++++++-
 .../fast/LazyBinaryDeserializeRead.java         |  6 ++-
 .../fast/LazyBinarySerializeWrite.java          |  5 +-
 .../hive/serde2/typeinfo/HiveDecimalUtils.java  | 35 +-------------
 12 files changed, 86 insertions(+), 98 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/29651cd3/common/src/java/org/apache/hadoop/hive/common/type/HiveDecimal.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/type/HiveDecimal.java b/common/src/java/org/apache/hadoop/hive/common/type/HiveDecimal.java
index f14fc2d..7d7fb28 100644
--- a/common/src/java/org/apache/hadoop/hive/common/type/HiveDecimal.java
+++ b/common/src/java/org/apache/hadoop/hive/common/type/HiveDecimal.java
@@ -296,6 +296,12 @@ public class HiveDecimal implements Comparable<HiveDecimal> {
       return null;
     }
 
+    // Minor optimization, avoiding creating new objects.
+    if (dec.precision() - dec.scale() <= maxPrecision - maxScale &&
+        dec.scale() <= maxScale) {
+      return dec;
+    }
+
     BigDecimal bd = enforcePrecisionScale(dec.bd, maxPrecision, maxScale);
     if (bd == null) {
       return null;

http://git-wip-us.apache.org/repos/asf/hive/blob/29651cd3/ql/src/java/org/apache/hadoop/hive/ql/io/orc/TreeReaderFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/TreeReaderFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/TreeReaderFactory.java
index 3ff6b14..9bfe268 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/TreeReaderFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/TreeReaderFactory.java
@@ -47,7 +47,6 @@ import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable;
 import org.apache.hadoop.hive.serde2.io.ShortWritable;
 import org.apache.hadoop.hive.serde2.io.TimestampWritable;
-import org.apache.hadoop.hive.serde2.typeinfo.HiveDecimalUtils;
 import org.apache.hadoop.io.BooleanWritable;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.FloatWritable;
@@ -1261,9 +1260,10 @@ public class TreeReaderFactory {
         } else {
           result = (HiveDecimalWritable) previous;
         }
-        result.set(HiveDecimal.create(SerializationUtils.readBigInteger(valueStream),
-            (int) scaleReader.next()));
-        return HiveDecimalUtils.enforcePrecisionScale(result, precision, scale);
+        result.set(HiveDecimal.create(SerializationUtils.readBigInteger
+                (valueStream), (int) scaleReader.next()));
+        return HiveDecimalWritable.enforcePrecisionScale(result, precision,
+            scale);
       }
       return null;
     }
@@ -1289,7 +1289,7 @@ public class TreeReaderFactory {
           BigInteger bInt = SerializationUtils.readBigInteger(valueStream);
           short scaleInData = (short) scaleReader.next();
           HiveDecimal dec = HiveDecimal.create(bInt, scaleInData);
-          dec = HiveDecimalUtils.enforcePrecisionScale(dec, precision, scale);
+          dec = HiveDecimal.enforcePrecisionScale(dec, precision, scale);
           result.set(0, dec);
         }
       } else {
@@ -1301,7 +1301,7 @@ public class TreeReaderFactory {
             BigInteger bInt = SerializationUtils.readBigInteger(valueStream);
             short scaleInData = (short) scratchScaleVector.vector[i];
             HiveDecimal dec = HiveDecimal.create(bInt, scaleInData);
-            dec = HiveDecimalUtils.enforcePrecisionScale(dec, precision, scale);
+            dec = HiveDecimal.enforcePrecisionScale(dec, precision, scale);
             result.set(i, dec);
           }
         }

http://git-wip-us.apache.org/repos/asf/hive/blob/29651cd3/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBridge.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBridge.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBridge.java
index e471285..6098ddd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBridge.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBridge.java
@@ -22,7 +22,6 @@ import java.io.Serializable;
 import java.lang.reflect.Method;
 import java.util.ArrayList;
 
-import org.apache.hadoop.hive.common.JavaUtils;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
 import org.apache.hadoop.hive.ql.exec.UDF;
@@ -34,7 +33,6 @@ import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions;
-import org.apache.hadoop.hive.serde2.typeinfo.HiveDecimalUtils;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
 
@@ -185,8 +183,10 @@ public class GenericUDFBridge extends GenericUDF implements Serializable {
     // For non-generic UDF, type info isn't available. This poses a problem for Hive Decimal.
     // If the returned value is HiveDecimal, we assume maximum precision/scale.
     if (result != null && result instanceof HiveDecimalWritable) {
-      result = HiveDecimalUtils.enforcePrecisionScale((HiveDecimalWritable) result,
-          HiveDecimal.SYSTEM_DEFAULT_PRECISION, HiveDecimal.SYSTEM_DEFAULT_SCALE);
+      result = HiveDecimalWritable.enforcePrecisionScale
+          ((HiveDecimalWritable) result,
+              HiveDecimal.SYSTEM_DEFAULT_PRECISION,
+              HiveDecimal.SYSTEM_DEFAULT_SCALE);
     }
 
     return result;

http://git-wip-us.apache.org/repos/asf/hive/blob/29651cd3/serde/src/java/org/apache/hadoop/hive/serde2/SerDeUtils.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/SerDeUtils.java b/serde/src/java/org/apache/hadoop/hive/serde2/SerDeUtils.java
index c65174e..192e814 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/SerDeUtils.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/SerDeUtils.java
@@ -27,6 +27,8 @@ import java.util.Properties;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
+import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth;
 import org.apache.hadoop.hive.serde2.AbstractSerDe;
 import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.MapObjectInspector;
@@ -44,6 +46,8 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.DoubleObjectInspe
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.FloatObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveCharObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveDecimalObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveIntervalDayTimeObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveIntervalYearMonthObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveVarcharObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.IntObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspector;
@@ -201,7 +205,6 @@ public final class SerDeUtils {
 
 
   static void buildJSONString(StringBuilder sb, Object o, ObjectInspector oi, String nullStr) {
-
     switch (oi.getCategory()) {
     case PRIMITIVE: {
       PrimitiveObjectInspector poi = (PrimitiveObjectInspector) oi;
@@ -284,6 +287,15 @@ public final class SerDeUtils {
           sb.append(((HiveDecimalObjectInspector) oi).getPrimitiveJavaObject(o));
           break;
         }
+        case INTERVAL_YEAR_MONTH: {
+          sb.append(((HiveIntervalYearMonthObjectInspector) oi).getPrimitiveJavaObject(o));
+          break;
+        }
+        case INTERVAL_DAY_TIME: {
+          sb.append(((HiveIntervalDayTimeObjectInspector) oi).getPrimitiveJavaObject(o));
+          break;
+        }
+
         default:
           throw new RuntimeException("Unknown primitive type: "
               + poi.getPrimitiveCategory());

http://git-wip-us.apache.org/repos/asf/hive/blob/29651cd3/serde/src/java/org/apache/hadoop/hive/serde2/io/HiveDecimalWritable.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/io/HiveDecimalWritable.java b/serde/src/java/org/apache/hadoop/hive/serde2/io/HiveDecimalWritable.java
index 885828a..0578d24 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/io/HiveDecimalWritable.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/io/HiveDecimalWritable.java
@@ -24,13 +24,8 @@ import java.math.BigInteger;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hive.common.type.Decimal128;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
-import org.apache.hadoop.hive.serde2.ByteStream.Output;
-import org.apache.hadoop.hive.serde2.ByteStream.RandomAccessOutput;
-import org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryUtils;
-import org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryUtils.VInt;
-import org.apache.hadoop.hive.serde2.typeinfo.HiveDecimalUtils;
+
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.io.WritableUtils;
 
@@ -41,8 +36,6 @@ public class HiveDecimalWritable implements WritableComparable<HiveDecimalWritab
   private byte[] internalStorage = new byte[0];
   private int scale;
 
-  private final VInt vInt = new VInt(); // reusable integer
-
   public HiveDecimalWritable() {
   }
 
@@ -83,18 +76,6 @@ public class HiveDecimalWritable implements WritableComparable<HiveDecimalWritab
     this.scale = scale;
   }
 
-  public void setFromBytes(byte[] bytes, int offset, int length) {
-    LazyBinaryUtils.readVInt(bytes, offset, vInt);
-    scale = vInt.value;
-    offset += vInt.length;
-    LazyBinaryUtils.readVInt(bytes, offset, vInt);
-    offset += vInt.length;
-    if (internalStorage.length != vInt.value) {
-      internalStorage = new byte[vInt.value];
-    }
-    System.arraycopy(bytes, offset, internalStorage, 0, vInt.value);
-  }
-
   public HiveDecimal getHiveDecimal() {
     return HiveDecimal.create(new BigInteger(internalStorage), scale);
   }
@@ -107,7 +88,8 @@ public class HiveDecimalWritable implements WritableComparable<HiveDecimalWritab
    * @return HiveDecimal instance
    */
   public HiveDecimal getHiveDecimal(int maxPrecision, int maxScale) {
-     return HiveDecimalUtils.enforcePrecisionScale(HiveDecimal.create(new BigInteger(internalStorage), scale),
+     return HiveDecimal.enforcePrecisionScale(HiveDecimal.
+             create(new BigInteger(internalStorage), scale),
          maxPrecision, maxScale);
   }
 
@@ -133,20 +115,6 @@ public class HiveDecimalWritable implements WritableComparable<HiveDecimalWritab
     return getHiveDecimal().compareTo(that.getHiveDecimal());
   }
 
-  public static void writeToByteStream(Decimal128 dec, Output byteStream) {
-    HiveDecimal hd = HiveDecimal.create(dec.toBigDecimal());
-    LazyBinaryUtils.writeVInt(byteStream, hd.scale());
-    byte[] bytes = hd.unscaledValue().toByteArray();
-    LazyBinaryUtils.writeVInt(byteStream, bytes.length);
-    byteStream.write(bytes, 0, bytes.length);
-  }
-
-  public void writeToByteStream(RandomAccessOutput byteStream) {
-    LazyBinaryUtils.writeVInt(byteStream, scale);
-    LazyBinaryUtils.writeVInt(byteStream, internalStorage.length);
-    byteStream.write(internalStorage, 0, internalStorage.length);
-  }
-
   @Override
   public String toString() {
     return getHiveDecimal().toString();
@@ -190,4 +158,17 @@ public class HiveDecimalWritable implements WritableComparable<HiveDecimalWritab
   public int getScale() {
     return scale;
   }
+
+  public static
+  HiveDecimalWritable enforcePrecisionScale(HiveDecimalWritable writable,
+                                            int precision, int scale) {
+    if (writable == null) {
+      return null;
+    }
+
+    HiveDecimal dec =
+        HiveDecimal.enforcePrecisionScale(writable.getHiveDecimal(), precision,
+            scale);
+    return dec == null ? null : new HiveDecimalWritable(dec);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/29651cd3/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyHiveDecimal.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyHiveDecimal.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyHiveDecimal.java
index fcf1ac6..b8b1f59 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyHiveDecimal.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyHiveDecimal.java
@@ -28,7 +28,6 @@ import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive.LazyHiveDecimalObjectInspector;
 import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.HiveDecimalUtils;
 import org.apache.hadoop.io.Text;
 
 public class LazyHiveDecimal extends LazyPrimitive<LazyHiveDecimalObjectInspector, HiveDecimalWritable> {
@@ -89,7 +88,7 @@ public class LazyHiveDecimal extends LazyPrimitive<LazyHiveDecimalObjectInspecto
   }
 
   private HiveDecimal enforcePrecisionScale(HiveDecimal dec) {
-    return HiveDecimalUtils.enforcePrecisionScale(dec, precision, scale);
+    return HiveDecimal.enforcePrecisionScale(dec, precision, scale);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/29651cd3/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleDeserializeRead.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleDeserializeRead.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleDeserializeRead.java
index 8c5b0b3..7588106 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleDeserializeRead.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleDeserializeRead.java
@@ -30,8 +30,7 @@ import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
 import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth;
 import org.apache.hadoop.hive.serde2.fast.DeserializeRead;
-import org.apache.hadoop.hive.serde2.fast.DeserializeRead.ReadIntervalDayTimeResults;
-import org.apache.hadoop.hive.serde2.fast.DeserializeRead.ReadIntervalYearMonthResults;
+
 import org.apache.hadoop.hive.serde2.io.DateWritable;
 import org.apache.hadoop.hive.serde2.io.HiveCharWritable;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
@@ -44,17 +43,11 @@ import org.apache.hadoop.hive.serde2.lazy.LazyBinary;
 import org.apache.hadoop.hive.serde2.lazy.LazyByte;
 import org.apache.hadoop.hive.serde2.lazy.LazyInteger;
 import org.apache.hadoop.hive.serde2.lazy.LazyLong;
-import org.apache.hadoop.hive.serde2.lazy.LazyPrimitive;
 import org.apache.hadoop.hive.serde2.lazy.LazySerDeParameters;
 import org.apache.hadoop.hive.serde2.lazy.LazyShort;
 import org.apache.hadoop.hive.serde2.lazy.LazyUtils;
-import org.apache.hadoop.hive.serde2.lazy.objectinspector.LazySimpleStructObjectInspector;
-import org.apache.hadoop.hive.serde2.lazy.objectinspector.primitive.LazyObjectInspectorParameters;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
-import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
 import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.HiveDecimalUtils;
+import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo;
 import org.apache.hadoop.io.Text;
@@ -494,7 +487,8 @@ public class LazySimpleDeserializeRead implements DeserializeRead {
         saveDecimalTypeInfo = (DecimalTypeInfo) primitiveTypeInfos[fieldIndex];
         int precision = saveDecimalTypeInfo.getPrecision();
         int scale = saveDecimalTypeInfo.getScale();
-        saveDecimal = HiveDecimalUtils.enforcePrecisionScale(saveDecimal, precision, scale);
+        saveDecimal = HiveDecimal.enforcePrecisionScale(saveDecimal, precision,
+            scale);
         if (saveDecimal == null) {
           LOG.debug("Data not in the HiveDecimal data type range so converted to null. Given data is :"
               + byteData);
@@ -1059,4 +1053,4 @@ public class LazySimpleDeserializeRead implements DeserializeRead {
     return 0;
   }
 
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/29651cd3/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryHiveDecimal.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryHiveDecimal.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryHiveDecimal.java
index e56e2ca..f8469a7 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryHiveDecimal.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryHiveDecimal.java
@@ -44,7 +44,7 @@ public class LazyBinaryHiveDecimal extends
 
   @Override
   public void init(ByteArrayRef bytes, int start, int length) {
-    data.setFromBytes(bytes.getData(), start, length);
+    LazyBinarySerDe.setFromBytes(bytes.getData(), start, length, data);
     HiveDecimal dec = data.getHiveDecimal(precision, scale);
     data = dec == null ? null : new HiveDecimalWritable(dec);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/29651cd3/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe.java
index a5dc5d8..41fe98a 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe.java
@@ -316,6 +316,30 @@ public class LazyBinarySerDe extends AbstractSerDe {
     LazyBinaryUtils.writeVInt(byteStream, date.getDays());
   }
 
+  public static void setFromBytes(byte[] bytes, int offset, int length,
+                                  HiveDecimalWritable dec) {
+    LazyBinaryUtils.VInt vInt = new LazyBinaryUtils.VInt();
+    LazyBinaryUtils.readVInt(bytes, offset, vInt);
+    int scale = vInt.value;
+    offset += vInt.length;
+    LazyBinaryUtils.readVInt(bytes, offset, vInt);
+    offset += vInt.length;
+    byte[] internalStorage = dec.getInternalStorage();
+    if (internalStorage.length != vInt.value) {
+      internalStorage = new byte[vInt.value];
+    }
+    System.arraycopy(bytes, offset, internalStorage, 0, vInt.value);
+    dec.set(internalStorage, scale);
+  }
+
+  public static void writeToByteStream(RandomAccessOutput byteStream,
+                                       HiveDecimalWritable dec) {
+    LazyBinaryUtils.writeVInt(byteStream, dec.getScale());
+    byte[] internalStorage = dec.getInternalStorage();
+    LazyBinaryUtils.writeVInt(byteStream, internalStorage.length);
+    byteStream.write(internalStorage, 0, internalStorage.length);
+  }
+
   /**
    * A recursive function that serialize an object to a byte buffer based on its
    * object inspector.
@@ -457,7 +481,7 @@ public class LazyBinarySerDe extends AbstractSerDe {
         if (t == null) {
           return;
         }
-        t.writeToByteStream(byteStream);
+        writeToByteStream(byteStream, t);
         return;
       }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/29651cd3/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinaryDeserializeRead.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinaryDeserializeRead.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinaryDeserializeRead.java
index a18e8b8..1f3806e 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinaryDeserializeRead.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinaryDeserializeRead.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hive.serde2.io.HiveIntervalDayTimeWritable;
 import org.apache.hadoop.hive.serde2.io.HiveIntervalYearMonthWritable;
 import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable;
 import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe;
 import org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryUtils;
 import org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryUtils.VInt;
 import org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryUtils.VLong;
@@ -913,7 +914,8 @@ public class LazyBinaryDeserializeRead implements DeserializeRead {
     if (tempHiveDecimalWritable == null) {
       tempHiveDecimalWritable = new HiveDecimalWritable();
     }
-    tempHiveDecimalWritable.setFromBytes(bytes, saveStart, length);
+    LazyBinarySerDe.setFromBytes(bytes, saveStart, length,
+        tempHiveDecimalWritable);
 
     saveDecimalTypeInfo = (DecimalTypeInfo) primitiveTypeInfos[fieldIndex];
 
@@ -939,4 +941,4 @@ public class LazyBinaryDeserializeRead implements DeserializeRead {
     // Now return whether it is NULL or NOT NULL.
     return (saveDecimal == null);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/29651cd3/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinarySerializeWrite.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinarySerializeWrite.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinarySerializeWrite.java
index e0d9c0a..253b514 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinarySerializeWrite.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinarySerializeWrite.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.serde2.io.HiveIntervalDayTimeWritable;
 import org.apache.hadoop.hive.serde2.io.HiveIntervalYearMonthWritable;
 import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe;
 import org.apache.hadoop.hive.serde2.lazybinary.LazyBinaryUtils;
 import org.apache.hadoop.hive.serde2.fast.SerializeWrite;
 import org.apache.hive.common.util.DateUtils;
@@ -733,7 +734,7 @@ public class LazyBinarySerializeWrite implements SerializeWrite {
       hiveDecimalWritable = new HiveDecimalWritable();
     }
     hiveDecimalWritable.set(v);
-    hiveDecimalWritable.writeToByteStream(output);
+    LazyBinarySerDe.writeToByteStream(output, hiveDecimalWritable);
 
     fieldIndex++;
 
@@ -742,4 +743,4 @@ public class LazyBinarySerializeWrite implements SerializeWrite {
       output.writeByte(nullOffset, nullByte);
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/29651cd3/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/HiveDecimalUtils.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/HiveDecimalUtils.java b/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/HiveDecimalUtils.java
index aa9e37a..cdd20bb 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/HiveDecimalUtils.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/HiveDecimalUtils.java
@@ -18,35 +18,14 @@
 
 package org.apache.hadoop.hive.serde2.typeinfo;
 
-import java.math.BigDecimal;
-
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
-import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
 
 public class HiveDecimalUtils {
 
   public static HiveDecimal enforcePrecisionScale(HiveDecimal dec, DecimalTypeInfo typeInfo) {
-    return enforcePrecisionScale(dec, typeInfo.precision(), typeInfo.scale());
-  }
-
-  public static HiveDecimal enforcePrecisionScale(HiveDecimal dec,int maxPrecision, int maxScale) {
-    if (dec == null) {
-      return null;
-    }
-
-    // Minor optimization, avoiding creating new objects.
-    if (dec.precision() - dec.scale() <= maxPrecision - maxScale && dec.scale() <= maxScale) {
-      return dec;
-    }
-
-    BigDecimal bd = HiveDecimal.enforcePrecisionScale(dec.bigDecimalValue(),
-        maxPrecision, maxScale);
-    if (bd == null) {
-      return null;
-    }
-
-    return HiveDecimal.create(bd);
+    return HiveDecimal.enforcePrecisionScale(dec, typeInfo.precision(),
+        typeInfo.scale());
   }
 
   public static HiveDecimalWritable enforcePrecisionScale(HiveDecimalWritable writable,
@@ -59,16 +38,6 @@ public class HiveDecimalUtils {
     return dec == null ? null : new HiveDecimalWritable(dec);
   }
 
-  public static HiveDecimalWritable enforcePrecisionScale(HiveDecimalWritable writable,
-      int precision, int scale) {
-    if (writable == null) {
-      return null;
-    }
-
-    HiveDecimal dec = enforcePrecisionScale(writable.getHiveDecimal(), precision, scale);
-    return dec == null ? null : new HiveDecimalWritable(dec);
-  }
-
   public static void validateParameter(int precision, int scale) {
     if (precision < 1 || precision > HiveDecimal.MAX_PRECISION) {
       throw new IllegalArgumentException("Decimal precision out of allowed range [1," +


[35/43] hive git commit: HIVE-11253. Move SearchArgument and VectorizedRowBatch classes to storage-api. (omalley reviewed by prasanthj)

Posted by xu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/9ae70cb4/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentImpl.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentImpl.java b/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentImpl.java
new file mode 100644
index 0000000..d27ac16
--- /dev/null
+++ b/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentImpl.java
@@ -0,0 +1,687 @@
+/**
+ * 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.io.sarg;
+
+import java.sql.Timestamp;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.codec.binary.Base64;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * The implementation of SearchArguments.
+ */
+final class SearchArgumentImpl implements SearchArgument {
+  public static final Log LOG = LogFactory.getLog(SearchArgumentImpl.class);
+
+  static final class PredicateLeafImpl implements PredicateLeaf {
+    private final Operator operator;
+    private final Type type;
+    private final String columnName;
+    private final Object literal;
+    private final List<Object> literalList;
+
+    // Used by kryo
+    @SuppressWarnings("unused")
+    PredicateLeafImpl() {
+      operator = null;
+      type = null;
+      columnName = null;
+      literal = null;
+      literalList = null;
+    }
+
+    PredicateLeafImpl(Operator operator,
+                      Type type,
+                      String columnName,
+                      Object literal,
+                      List<Object> literalList) {
+      this.operator = operator;
+      this.type = type;
+      this.columnName = columnName;
+      this.literal = literal;
+      if (literal != null) {
+        if (literal.getClass() != type.getValueClass()) {
+          throw new IllegalArgumentException("Wrong value class " +
+              literal.getClass().getName() + " for " + type + "." + operator +
+              " leaf");
+        }
+      }
+      this.literalList = literalList;
+      if (literalList != null) {
+        Class valueCls = type.getValueClass();
+        for(Object lit: literalList) {
+          if (lit != null && lit.getClass() != valueCls) {
+            throw new IllegalArgumentException("Wrong value class item " +
+                lit.getClass().getName() + " for " + type + "." + operator +
+                " leaf");
+          }
+        }
+      }
+    }
+
+    @Override
+    public Operator getOperator() {
+      return operator;
+    }
+
+    @Override
+    public Type getType(){
+      return type;
+    }
+
+    @Override
+    public String getColumnName() {
+      return columnName;
+    }
+
+    @Override
+    public Object getLiteral() {
+      // To get around a kryo 2.22 bug while deserialize a Timestamp into Date
+      // (https://github.com/EsotericSoftware/kryo/issues/88)
+      // When we see a Date, convert back into Timestamp
+      if (literal instanceof java.util.Date) {
+        return new Timestamp(((java.util.Date)literal).getTime());
+      }
+      return literal;
+    }
+
+    @Override
+    public List<Object> getLiteralList() {
+      return literalList;
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder buffer = new StringBuilder();
+      buffer.append('(');
+      buffer.append(operator);
+      buffer.append(' ');
+      buffer.append(columnName);
+      if (literal != null) {
+        buffer.append(' ');
+        buffer.append(literal);
+      } else if (literalList != null) {
+        for(Object lit: literalList) {
+          buffer.append(' ');
+          buffer.append(lit == null ? "null" : lit.toString());
+        }
+      }
+      buffer.append(')');
+      return buffer.toString();
+    }
+
+    private static boolean isEqual(Object left, Object right) {
+
+      return left == right ||
+          (left != null && right != null && left.equals(right));
+    }
+
+    @Override
+    public boolean equals(Object other) {
+      if (other == null || other.getClass() != getClass()) {
+        return false;
+      } else if (other == this) {
+        return true;
+      } else {
+        PredicateLeafImpl o = (PredicateLeafImpl) other;
+        return operator == o.operator &&
+            type == o.type &&
+            columnName.equals(o.columnName) &&
+            isEqual(literal, o.literal) &&
+            isEqual(literalList, o.literalList);
+      }
+    }
+
+    @Override
+    public int hashCode() {
+      return operator.hashCode() +
+             type.hashCode() * 17 +
+             columnName.hashCode() * 3 * 17+
+             (literal == null ? 0 : literal.hashCode()) * 101 * 3 * 17 +
+             (literalList == null ? 0 : literalList.hashCode()) *
+                 103 * 101 * 3 * 17;
+    }
+  }
+
+
+  private final List<PredicateLeaf> leaves;
+  private final ExpressionTree expression;
+
+  SearchArgumentImpl(ExpressionTree expression, List<PredicateLeaf> leaves) {
+    this.expression = expression;
+    this.leaves = leaves;
+  }
+
+  // Used by kyro
+  @SuppressWarnings("unused")
+  SearchArgumentImpl() {
+        leaves = null;
+        expression = null;
+  }
+
+  @Override
+  public List<PredicateLeaf> getLeaves() {
+    return leaves;
+  }
+
+  @Override
+  public TruthValue evaluate(TruthValue[] leaves) {
+    return expression == null ? TruthValue.YES : expression.evaluate(leaves);
+  }
+
+  @Override
+  public ExpressionTree getExpression() {
+    return expression;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder buffer = new StringBuilder();
+    for(int i=0; i < leaves.size(); ++i) {
+      buffer.append("leaf-");
+      buffer.append(i);
+      buffer.append(" = ");
+      buffer.append(leaves.get(i).toString());
+      buffer.append('\n');
+    }
+    buffer.append("expr = ");
+    buffer.append(expression);
+    return buffer.toString();
+  }
+
+  static class BuilderImpl implements Builder {
+
+    // max threshold for CNF conversion. having >8 elements in andList will be
+    // converted to maybe
+    private static final int CNF_COMBINATIONS_THRESHOLD = 256;
+
+    private final Deque<ExpressionTree> currentTree =
+        new ArrayDeque<ExpressionTree>();
+    private final Map<PredicateLeaf, Integer> leaves =
+        new HashMap<PredicateLeaf, Integer>();
+    private final ExpressionTree root =
+        new ExpressionTree(ExpressionTree.Operator.AND);
+    {
+      currentTree.add(root);
+    }
+
+    @Override
+    public Builder startOr() {
+      ExpressionTree node = new ExpressionTree(ExpressionTree.Operator.OR);
+      currentTree.getFirst().getChildren().add(node);
+      currentTree.addFirst(node);
+      return this;
+    }
+
+    @Override
+    public Builder startAnd() {
+      ExpressionTree node = new ExpressionTree(ExpressionTree.Operator.AND);
+      currentTree.getFirst().getChildren().add(node);
+      currentTree.addFirst(node);
+      return this;
+    }
+
+    @Override
+    public Builder startNot() {
+      ExpressionTree node = new ExpressionTree(ExpressionTree.Operator.NOT);
+      currentTree.getFirst().getChildren().add(node);
+      currentTree.addFirst(node);
+      return this;
+    }
+
+    @Override
+    public Builder end() {
+      ExpressionTree current = currentTree.removeFirst();
+      if (current.getChildren().size() == 0) {
+        throw new IllegalArgumentException("Can't create expression " + root +
+            " with no children.");
+      }
+      if (current.getOperator() == ExpressionTree.Operator.NOT &&
+          current.getChildren().size() != 1) {
+        throw new IllegalArgumentException("Can't create not expression " +
+            current + " with more than 1 child.");
+      }
+      return this;
+    }
+
+    private int addLeaf(PredicateLeaf leaf) {
+      Integer result = leaves.get(leaf);
+      if (result == null) {
+        int id = leaves.size();
+        leaves.put(leaf, id);
+        return id;
+      } else {
+        return result;
+      }
+    }
+
+    @Override
+    public Builder lessThan(String column, PredicateLeaf.Type type,
+                            Object literal) {
+      ExpressionTree parent = currentTree.getFirst();
+      if (column == null || literal == null) {
+        parent.getChildren().add(new ExpressionTree(TruthValue.YES_NO_NULL));
+      } else {
+        PredicateLeaf leaf =
+            new PredicateLeafImpl(PredicateLeaf.Operator.LESS_THAN,
+                type, column, literal, null);
+        parent.getChildren().add(new ExpressionTree(addLeaf(leaf)));
+      }
+      return this;
+    }
+
+    @Override
+    public Builder lessThanEquals(String column, PredicateLeaf.Type type,
+                                  Object literal) {
+      ExpressionTree parent = currentTree.getFirst();
+      if (column == null || literal == null) {
+        parent.getChildren().add(new ExpressionTree(TruthValue.YES_NO_NULL));
+      } else {
+        PredicateLeaf leaf =
+            new PredicateLeafImpl(PredicateLeaf.Operator.LESS_THAN_EQUALS,
+                type, column, literal, null);
+        parent.getChildren().add(new ExpressionTree(addLeaf(leaf)));
+      }
+      return this;
+    }
+
+    @Override
+    public Builder equals(String column, PredicateLeaf.Type type,
+                          Object literal) {
+      ExpressionTree parent = currentTree.getFirst();
+      if (column == null || literal == null) {
+        parent.getChildren().add(new ExpressionTree(TruthValue.YES_NO_NULL));
+      } else {
+        PredicateLeaf leaf =
+            new PredicateLeafImpl(PredicateLeaf.Operator.EQUALS,
+                type, column, literal, null);
+        parent.getChildren().add(new ExpressionTree(addLeaf(leaf)));
+      }
+      return this;
+    }
+
+    @Override
+    public Builder nullSafeEquals(String column, PredicateLeaf.Type type,
+                                  Object literal) {
+      ExpressionTree parent = currentTree.getFirst();
+      if (column == null || literal == null) {
+        parent.getChildren().add(new ExpressionTree(TruthValue.YES_NO_NULL));
+      } else {
+        PredicateLeaf leaf =
+            new PredicateLeafImpl(PredicateLeaf.Operator.NULL_SAFE_EQUALS,
+                type, column, literal, null);
+        parent.getChildren().add(new ExpressionTree(addLeaf(leaf)));
+      }
+      return this;
+    }
+
+    @Override
+    public Builder in(String column, PredicateLeaf.Type type,
+                      Object... literal) {
+      ExpressionTree parent = currentTree.getFirst();
+      if (column  == null || literal == null) {
+        parent.getChildren().add(new ExpressionTree(TruthValue.YES_NO_NULL));
+      } else {
+        if (literal.length == 0) {
+          throw new IllegalArgumentException("Can't create in expression with "
+              + "no arguments");
+        }
+        List<Object> argList = new ArrayList<Object>();
+        argList.addAll(Arrays.asList(literal));
+
+        PredicateLeaf leaf =
+            new PredicateLeafImpl(PredicateLeaf.Operator.IN,
+                type, column, null, argList);
+        parent.getChildren().add(new ExpressionTree(addLeaf(leaf)));
+      }
+      return this;
+    }
+
+    @Override
+    public Builder isNull(String column, PredicateLeaf.Type type) {
+      ExpressionTree parent = currentTree.getFirst();
+      if (column == null) {
+        parent.getChildren().add(new ExpressionTree(TruthValue.YES_NO_NULL));
+      } else {
+        PredicateLeaf leaf =
+            new PredicateLeafImpl(PredicateLeaf.Operator.IS_NULL,
+                type, column, null, null);
+        parent.getChildren().add(new ExpressionTree(addLeaf(leaf)));
+      }
+      return this;
+    }
+
+    @Override
+    public Builder between(String column, PredicateLeaf.Type type, Object lower,
+                           Object upper) {
+      ExpressionTree parent = currentTree.getFirst();
+      if (column == null || lower == null || upper == null) {
+        parent.getChildren().add(new ExpressionTree(TruthValue.YES_NO_NULL));
+      } else {
+        List<Object> argList = new ArrayList<Object>();
+        argList.add(lower);
+        argList.add(upper);
+        PredicateLeaf leaf =
+            new PredicateLeafImpl(PredicateLeaf.Operator.BETWEEN,
+                type, column, null, argList);
+        parent.getChildren().add(new ExpressionTree(addLeaf(leaf)));
+      }
+      return this;
+    }
+
+    @Override
+    public Builder literal(TruthValue truth) {
+      ExpressionTree parent = currentTree.getFirst();
+      parent.getChildren().add(new ExpressionTree(truth));
+      return this;
+    }
+
+    /**
+     * Recursively explore the tree to find the leaves that are still reachable
+     * after optimizations.
+     * @param tree the node to check next
+     * @param next the next available leaf id
+     * @param leafReorder
+     * @return the next available leaf id
+     */
+    static int compactLeaves(ExpressionTree tree, int next, int[] leafReorder) {
+      if (tree.getOperator() == ExpressionTree.Operator.LEAF) {
+        int oldLeaf = tree.getLeaf();
+        if (leafReorder[oldLeaf] == -1) {
+          leafReorder[oldLeaf] = next++;
+        }
+      } else if (tree.getChildren() != null){
+        for(ExpressionTree child: tree.getChildren()) {
+          next = compactLeaves(child, next, leafReorder);
+        }
+      }
+      return next;
+    }
+
+    /**
+     * Rewrite expression tree to update the leaves.
+     * @param root the root of the tree to fix
+     * @param leafReorder a map from old leaf ids to new leaf ids
+     * @return the fixed root
+     */
+    static ExpressionTree rewriteLeaves(ExpressionTree root,
+                                        int[] leafReorder) {
+      if (root.getOperator() == ExpressionTree.Operator.LEAF) {
+        return new ExpressionTree(leafReorder[root.getLeaf()]);
+      } else if (root.getChildren() != null){
+        List<ExpressionTree> children = root.getChildren();
+        for(int i=0; i < children.size(); ++i) {
+          children.set(i, rewriteLeaves(children.get(i), leafReorder));
+        }
+      }
+      return root;
+    }
+
+    @Override
+    public SearchArgument build() {
+      if (currentTree.size() != 1) {
+        throw new IllegalArgumentException("Failed to end " +
+            currentTree.size() + " operations.");
+      }
+      ExpressionTree optimized = pushDownNot(root);
+      optimized = foldMaybe(optimized);
+      optimized = flatten(optimized);
+      optimized = convertToCNF(optimized);
+      optimized = flatten(optimized);
+      int leafReorder[] = new int[leaves.size()];
+      Arrays.fill(leafReorder, -1);
+      int newLeafCount = compactLeaves(optimized, 0, leafReorder);
+      optimized = rewriteLeaves(optimized, leafReorder);
+      ArrayList<PredicateLeaf> leafList = new ArrayList<>(newLeafCount);
+      // expand list to correct size
+      for(int i=0; i < newLeafCount; ++i) {
+        leafList.add(null);
+      }
+      // build the new list
+      for(Map.Entry<PredicateLeaf, Integer> elem: leaves.entrySet()) {
+        int newLoc = leafReorder[elem.getValue()];
+        if (newLoc != -1) {
+          leafList.set(newLoc, elem.getKey());
+        }
+      }
+      return new SearchArgumentImpl(optimized, leafList);
+    }
+
+    /**
+     * Push the negations all the way to just before the leaves. Also remove
+     * double negatives.
+     * @param root the expression to normalize
+     * @return the normalized expression, which may share some or all of the
+     * nodes of the original expression.
+     */
+    static ExpressionTree pushDownNot(ExpressionTree root) {
+      if (root.getOperator() == ExpressionTree.Operator.NOT) {
+        ExpressionTree child = root.getChildren().get(0);
+        switch (child.getOperator()) {
+          case NOT:
+            return pushDownNot(child.getChildren().get(0));
+          case CONSTANT:
+            return  new ExpressionTree(child.getConstant().not());
+          case AND:
+            root = new ExpressionTree(ExpressionTree.Operator.OR);
+            for(ExpressionTree kid: child.getChildren()) {
+              root.getChildren().add(pushDownNot(new
+                  ExpressionTree(ExpressionTree.Operator.NOT, kid)));
+            }
+            break;
+          case OR:
+            root = new ExpressionTree(ExpressionTree.Operator.AND);
+            for(ExpressionTree kid: child.getChildren()) {
+              root.getChildren().add(pushDownNot(new ExpressionTree
+                  (ExpressionTree.Operator.NOT, kid)));
+            }
+            break;
+          // for leaf, we don't do anything
+          default:
+            break;
+        }
+      } else if (root.getChildren() != null) {
+        // iterate through children and push down not for each one
+        for(int i=0; i < root.getChildren().size(); ++i) {
+          root.getChildren().set(i, pushDownNot(root.getChildren().get(i)));
+        }
+      }
+      return root;
+    }
+
+    /**
+     * Remove MAYBE values from the expression. If they are in an AND operator,
+     * they are dropped. If they are in an OR operator, they kill their parent.
+     * This assumes that pushDownNot has already been called.
+     * @param expr The expression to clean up
+     * @return The cleaned up expression
+     */
+    static ExpressionTree foldMaybe(ExpressionTree expr) {
+      if (expr.getChildren() != null) {
+        for(int i=0; i < expr.getChildren().size(); ++i) {
+          ExpressionTree child = foldMaybe(expr.getChildren().get(i));
+          if (child.getConstant() == TruthValue.YES_NO_NULL) {
+            switch (expr.getOperator()) {
+              case AND:
+                expr.getChildren().remove(i);
+                i -= 1;
+                break;
+              case OR:
+                // a maybe will kill the or condition
+                return child;
+              default:
+                throw new IllegalStateException("Got a maybe as child of " +
+                    expr);
+            }
+          } else {
+            expr.getChildren().set(i, child);
+          }
+        }
+        if (expr.getChildren().isEmpty()) {
+          return new ExpressionTree(TruthValue.YES_NO_NULL);
+        }
+      }
+      return expr;
+    }
+
+    /**
+     * Converts multi-level ands and ors into single level ones.
+     * @param root the expression to flatten
+     * @return the flattened expression, which will always be root with
+     *   potentially modified children.
+     */
+    static ExpressionTree flatten(ExpressionTree root) {
+      if (root.getChildren() != null) {
+        // iterate through the index, so that if we add more children,
+        // they don't get re-visited
+        for(int i=0; i < root.getChildren().size(); ++i) {
+          ExpressionTree child = flatten(root.getChildren().get(i));
+          // do we need to flatten?
+          if (child.getOperator() == root.getOperator() &&
+              child.getOperator() != ExpressionTree.Operator.NOT) {
+            boolean first = true;
+            for(ExpressionTree grandkid: child.getChildren()) {
+              // for the first grandkid replace the original parent
+              if (first) {
+                first = false;
+                root.getChildren().set(i, grandkid);
+              } else {
+                root.getChildren().add(++i, grandkid);
+              }
+            }
+          } else {
+            root.getChildren().set(i, child);
+          }
+        }
+        // if we have a singleton AND or OR, just return the child
+        if ((root.getOperator() == ExpressionTree.Operator.OR ||
+            root.getOperator() == ExpressionTree.Operator.AND) &&
+            root.getChildren().size() == 1) {
+          return root.getChildren().get(0);
+        }
+      }
+      return root;
+    }
+
+    /**
+     * Generate all combinations of items on the andList. For each item on the
+     * andList, it generates all combinations of one child from each and
+     * expression. Thus, (and a b) (and c d) will be expanded to: (or a c)
+     * (or a d) (or b c) (or b d). If there are items on the nonAndList, they
+     * are added to each or expression.
+     * @param result a list to put the results onto
+     * @param andList a list of and expressions
+     * @param nonAndList a list of non-and expressions
+     */
+    private static void generateAllCombinations(List<ExpressionTree> result,
+                                                List<ExpressionTree> andList,
+                                                List<ExpressionTree> nonAndList
+    ) {
+      List<ExpressionTree> kids = andList.get(0).getChildren();
+      if (result.isEmpty()) {
+        for(ExpressionTree kid: kids) {
+          ExpressionTree or = new ExpressionTree(ExpressionTree.Operator.OR);
+          result.add(or);
+          for(ExpressionTree node: nonAndList) {
+            or.getChildren().add(new ExpressionTree(node));
+          }
+          or.getChildren().add(kid);
+        }
+      } else {
+        List<ExpressionTree> work = new ArrayList<ExpressionTree>(result);
+        result.clear();
+        for(ExpressionTree kid: kids) {
+          for(ExpressionTree or: work) {
+            ExpressionTree copy = new ExpressionTree(or);
+            copy.getChildren().add(kid);
+            result.add(copy);
+          }
+        }
+      }
+      if (andList.size() > 1) {
+        generateAllCombinations(result, andList.subList(1, andList.size()),
+            nonAndList);
+      }
+    }
+
+    /**
+     * Convert an expression so that the top level operator is AND with OR
+     * operators under it. This routine assumes that all of the NOT operators
+     * have been pushed to the leaves via pushdDownNot.
+     * @param root the expression
+     * @return the normalized expression
+     */
+    static ExpressionTree convertToCNF(ExpressionTree root) {
+      if (root.getChildren() != null) {
+        // convert all of the children to CNF
+        int size = root.getChildren().size();
+        for(int i=0; i < size; ++i) {
+          root.getChildren().set(i, convertToCNF(root.getChildren().get(i)));
+        }
+        if (root.getOperator() == ExpressionTree.Operator.OR) {
+          // a list of leaves that weren't under AND expressions
+          List<ExpressionTree> nonAndList = new ArrayList<ExpressionTree>();
+          // a list of AND expressions that we need to distribute
+          List<ExpressionTree> andList = new ArrayList<ExpressionTree>();
+          for(ExpressionTree child: root.getChildren()) {
+            if (child.getOperator() == ExpressionTree.Operator.AND) {
+              andList.add(child);
+            } else if (child.getOperator() == ExpressionTree.Operator.OR) {
+              // pull apart the kids of the OR expression
+              for(ExpressionTree grandkid: child.getChildren()) {
+                nonAndList.add(grandkid);
+              }
+            } else {
+              nonAndList.add(child);
+            }
+          }
+          if (!andList.isEmpty()) {
+            if (checkCombinationsThreshold(andList)) {
+              root = new ExpressionTree(ExpressionTree.Operator.AND);
+              generateAllCombinations(root.getChildren(), andList, nonAndList);
+            } else {
+              root = new ExpressionTree(TruthValue.YES_NO_NULL);
+            }
+          }
+        }
+      }
+      return root;
+    }
+
+    private static boolean checkCombinationsThreshold(List<ExpressionTree> andList) {
+      int numComb = 1;
+      for (ExpressionTree tree : andList) {
+        numComb *= tree.getChildren().size();
+        if (numComb > CNF_COMBINATIONS_THRESHOLD) {
+          return false;
+        }
+      }
+      return true;
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/9ae70cb4/storage-api/src/java/org/apache/hadoop/hive/serde2/io/HiveDecimalWritable.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/serde2/io/HiveDecimalWritable.java b/storage-api/src/java/org/apache/hadoop/hive/serde2/io/HiveDecimalWritable.java
new file mode 100644
index 0000000..9890771
--- /dev/null
+++ b/storage-api/src/java/org/apache/hadoop/hive/serde2/io/HiveDecimalWritable.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.serde2.io;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.math.BigInteger;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
+
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.WritableUtils;
+
+public class HiveDecimalWritable implements WritableComparable<HiveDecimalWritable> {
+
+  static final private Log LOG = LogFactory.getLog(HiveDecimalWritable.class);
+
+  private byte[] internalStorage = new byte[0];
+  private int scale;
+
+  public HiveDecimalWritable() {
+  }
+
+  public HiveDecimalWritable(String value) {
+    set(HiveDecimal.create(value));
+  }
+
+  public HiveDecimalWritable(byte[] bytes, int scale) {
+    set(bytes, scale);
+  }
+
+  public HiveDecimalWritable(HiveDecimalWritable writable) {
+    set(writable.getHiveDecimal());
+  }
+
+  public HiveDecimalWritable(HiveDecimal value) {
+    set(value);
+  }
+
+  public HiveDecimalWritable(long value) {
+    set((HiveDecimal.create(value)));
+  }
+
+  public void set(HiveDecimal value) {
+    set(value.unscaledValue().toByteArray(), value.scale());
+  }
+
+  public void set(HiveDecimal value, int maxPrecision, int maxScale) {
+    set(HiveDecimal.enforcePrecisionScale(value, maxPrecision, maxScale));
+  }
+
+  public void set(HiveDecimalWritable writable) {
+    set(writable.getHiveDecimal());
+  }
+
+  public void set(byte[] bytes, int scale) {
+    this.internalStorage = bytes;
+    this.scale = scale;
+  }
+
+  public HiveDecimal getHiveDecimal() {
+    return HiveDecimal.create(new BigInteger(internalStorage), scale);
+  }
+
+  /**
+   * Get a HiveDecimal instance from the writable and constraint it with maximum precision/scale.
+   *
+   * @param maxPrecision maximum precision
+   * @param maxScale maximum scale
+   * @return HiveDecimal instance
+   */
+  public HiveDecimal getHiveDecimal(int maxPrecision, int maxScale) {
+     return HiveDecimal.enforcePrecisionScale(HiveDecimal.
+             create(new BigInteger(internalStorage), scale),
+         maxPrecision, maxScale);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    scale = WritableUtils.readVInt(in);
+    int byteArrayLen = WritableUtils.readVInt(in);
+    if (internalStorage.length != byteArrayLen) {
+      internalStorage = new byte[byteArrayLen];
+    }
+    in.readFully(internalStorage);
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    WritableUtils.writeVInt(out, scale);
+    WritableUtils.writeVInt(out, internalStorage.length);
+    out.write(internalStorage);
+  }
+
+  @Override
+  public int compareTo(HiveDecimalWritable that) {
+    return getHiveDecimal().compareTo(that.getHiveDecimal());
+  }
+
+  @Override
+  public String toString() {
+    return getHiveDecimal().toString();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (this == other) {
+      return true;
+    }
+    if (other == null || getClass() != other.getClass()) {
+      return false;
+    }
+    HiveDecimalWritable bdw = (HiveDecimalWritable) other;
+
+    // 'equals' and 'compareTo' are not compatible with HiveDecimals. We want
+    // compareTo which returns true iff the numbers are equal (e.g.: 3.14 is
+    // the same as 3.140). 'Equals' returns true iff equal and the same scale
+    // is set in the decimals (e.g.: 3.14 is not the same as 3.140)
+    return getHiveDecimal().compareTo(bdw.getHiveDecimal()) == 0;
+  }
+
+  @Override
+  public int hashCode() {
+    return getHiveDecimal().hashCode();
+  }
+
+  /* (non-Javadoc)
+   * In order to update a Decimal128 fast (w/o allocation) we need to expose access to the
+   * internal storage bytes and scale.
+   * @return
+   */
+  public byte[] getInternalStorage() {
+    return internalStorage;
+  }
+
+  /* (non-Javadoc)
+   * In order to update a Decimal128 fast (w/o allocation) we need to expose access to the
+   * internal storage bytes and scale.
+   */
+  public int getScale() {
+    return scale;
+  }
+
+  public static
+  HiveDecimalWritable enforcePrecisionScale(HiveDecimalWritable writable,
+                                            int precision, int scale) {
+    if (writable == null) {
+      return null;
+    }
+
+    HiveDecimal dec =
+        HiveDecimal.enforcePrecisionScale(writable.getHiveDecimal(), precision,
+            scale);
+    return dec == null ? null : new HiveDecimalWritable(dec);
+  }
+}


[20/43] hive git commit: HIVE-11290 Cursor attributes %ISOPEN, %FOUND, %NOTFOUND and SYS_REFCURSOR variable (Dmitry Tolpeko via gates)

Posted by xu...@apache.org.
HIVE-11290 Cursor attributes %ISOPEN, %FOUND, %NOTFOUND and SYS_REFCURSOR variable (Dmitry Tolpeko via gates)


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

Branch: refs/heads/spark
Commit: 92e98858e742bbb669ccbf790a71a618c581df21
Parents: 2773b9d
Author: Alan Gates <ga...@hortonworks.com>
Authored: Fri Jul 24 10:43:45 2015 -0700
Committer: Alan Gates <ga...@hortonworks.com>
Committed: Fri Jul 24 10:43:45 2015 -0700

----------------------------------------------------------------------
 .../antlr4/org/apache/hive/hplsql/Hplsql.g4     | 38 ++++++++++--
 .../main/java/org/apache/hive/hplsql/Conn.java  |  6 ++
 .../main/java/org/apache/hive/hplsql/Exec.java  | 21 ++++++-
 .../java/org/apache/hive/hplsql/Expression.java | 28 +++++++++
 .../main/java/org/apache/hive/hplsql/Query.java | 55 +++++++++++++++++
 .../main/java/org/apache/hive/hplsql/Stmt.java  | 60 ++++++++++++------
 .../main/java/org/apache/hive/hplsql/Var.java   | 20 +++++-
 .../src/test/queries/db/cursor_attributes.sql   | 60 ++++++++++++++++++
 hplsql/src/test/queries/db/select_into.sql      | 17 +++++
 .../src/test/queries/db/set_current_schema.sql  |  6 ++
 hplsql/src/test/queries/db/sys_refcursor.sql    | 65 ++++++++++++++++++++
 hplsql/src/test/queries/db/use.sql              |  2 +
 .../test/results/db/cursor_attributes.out.txt   | 33 ++++++++++
 hplsql/src/test/results/db/select_into.out.txt  | 19 ++++++
 .../test/results/db/set_current_schema.out.txt  | 12 ++++
 .../src/test/results/db/sys_refcursor.out.txt   | 36 +++++++++++
 hplsql/src/test/results/db/use.out.txt          |  4 ++
 17 files changed, 453 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/92e98858/hplsql/src/main/antlr4/org/apache/hive/hplsql/Hplsql.g4
----------------------------------------------------------------------
diff --git a/hplsql/src/main/antlr4/org/apache/hive/hplsql/Hplsql.g4 b/hplsql/src/main/antlr4/org/apache/hive/hplsql/Hplsql.g4
index 70312b2..4104d13 100644
--- a/hplsql/src/main/antlr4/org/apache/hive/hplsql/Hplsql.g4
+++ b/hplsql/src/main/antlr4/org/apache/hive/hplsql/Hplsql.g4
@@ -98,7 +98,8 @@ expr_stmt :             // Standalone expression
      ;
 
 assignment_stmt :       // Assignment statement
-       T_SET? assignment_stmt_item (T_COMMA assignment_stmt_item)*  
+       T_SET set_session_option
+     | T_SET? assignment_stmt_item (T_COMMA assignment_stmt_item)*  
      ;
 
 assignment_stmt_item : 
@@ -145,9 +146,9 @@ declare_block :         // Declaration block
    
      
 declare_stmt_item :
-       declare_var_item 
+       declare_cursor_item
+     | declare_var_item 
      | declare_condition_item  
-     | declare_cursor_item
      | declare_handler_item
      | declare_temporary_table_item
      ;
@@ -254,6 +255,7 @@ dtype :                  // Data types
      | T_RESULT_SET_LOCATOR T_VARYING
      | T_SMALLINT
      | T_STRING
+     | T_SYS_REFCURSOR
      | T_TIMESTAMP
      | T_VARCHAR
      | T_VARCHAR2
@@ -391,7 +393,7 @@ map_object_stmt :
      ;
      
 open_stmt :             // OPEN cursor statement
-       T_OPEN L_ID (T_FOR (expr | select_stmt))?
+       T_OPEN L_ID (T_FOR (select_stmt | expr))?
      ;
 
 fetch_stmt :            // FETCH cursor statement
@@ -460,6 +462,14 @@ rollback_stmt :         // ROLLBACK statement
        T_ROLLBACK T_WORK?
      ;
      
+set_session_option :          
+       set_current_schema_option
+     ;
+
+set_current_schema_option :          
+       ((T_CURRENT? T_SCHEMA) | T_CURRENT_SCHEMA) T_EQUAL? expr
+     ;
+     
 signal_stmt :          // SIGNAL statement
        T_SIGNAL ident
      ;
@@ -675,6 +685,7 @@ bool_expr :                               // Boolean condition
 bool_expr_atom :
       bool_expr_unary
     | bool_expr_binary
+    | expr
     ;
     
 bool_expr_unary :
@@ -722,6 +733,7 @@ expr :
      | T_OPEN_P expr T_CLOSE_P 
      | expr_concat
      | expr_case
+     | expr_cursor_attribute
      | expr_agg_window_func
      | expr_spec_func
      | expr_func                          
@@ -777,6 +789,10 @@ expr_case_searched :
        T_CASE (T_WHEN bool_expr T_THEN expr)+ (T_ELSE expr)? T_END
      ;
      
+expr_cursor_attribute :
+      ident '%' (T_ISOPEN | T_FOUND | T_NOTFOUND)
+    ;
+     
 expr_agg_window_func :
        T_AVG T_OPEN_P expr_func_all_distinct? expr T_CLOSE_P expr_func_over_clause?
      | T_COUNT T_OPEN_P ((expr_func_all_distinct? expr) | '*') T_CLOSE_P expr_func_over_clause?
@@ -937,6 +953,7 @@ non_reserved_words :                      // Tokens that are not reserved words
      | T_CS
      | T_CURRENT 
      | T_CURRENT_DATE
+     | T_CURRENT_SCHEMA
      | T_CURRENT_TIMESTAMP
      | T_CURRENT_USER
      | T_CURSOR  
@@ -1007,6 +1024,7 @@ non_reserved_words :                      // Tokens that are not reserved words
      | T_INVOKER     
      | T_ITEMS     
      | T_IS    
+     | T_ISOPEN
      | T_JOIN     
      | T_KEY
      | T_KEYS
@@ -1033,7 +1051,8 @@ non_reserved_words :                      // Tokens that are not reserved words
      | T_MICROSECONDS
      | T_MIN
      | T_MULTISET
-     | T_NOT          
+     | T_NOT         
+     | T_NOTFOUND     
      // T_NULL reserved word       
      | T_NUMBER   
      | T_OBJECT     
@@ -1072,6 +1091,7 @@ non_reserved_words :                      // Tokens that are not reserved words
      | T_ROWS
      | T_ROW_COUNT
      | T_ROW_NUMBER
+     | T_SCHEMA
      | T_SECURITY
      | T_SEL          
      | T_SELECT       
@@ -1089,7 +1109,8 @@ non_reserved_words :                      // Tokens that are not reserved words
      | T_STRING    
      | T_SUBSTRING
      | T_SUM
-     | T_SYSDATE     
+     | T_SYSDATE 
+     | T_SYS_REFCURSOR     
      | T_TABLE
      | T_TEMPORARY
      | T_TERMINATED
@@ -1160,6 +1181,7 @@ T_CREATE          : C R E A T E ;
 T_CREATOR         : C R E A T O R ;
 T_CS              : C S;
 T_CURRENT         : C U R R E N T ;
+T_CURRENT_SCHEMA  : C U R R E N T '_' S C H E M A ;
 T_CURSOR          : C U R S O R ;
 T_DATE            : D A T E ;
 T_DAY             : D A Y ;
@@ -1225,6 +1247,7 @@ T_INTERSECT       : I N T E R S E C T ;
 T_INTO            : I N T O ;
 T_INVOKER         : I N V O K E R ;
 T_IS              : I S ;
+T_ISOPEN          : I S O P E N ;
 T_ITEMS           : I T E M S ; 
 T_JOIN            : J O I N ;
 T_KEY             : K E Y ;
@@ -1250,6 +1273,7 @@ T_MICROSECONDS    : M I C R O S E C O N D S;
 T_MIN             : M I N ;
 T_MULTISET        : M U L T I S E T ; 
 T_NOT             : N O T ;
+T_NOTFOUND        : N O T F O U N D ; 
 T_NULL            : N U L L ;
 T_NUMBER          : N U M B E R ;
 T_OBJECT          : O B J E C T ; 
@@ -1286,6 +1310,7 @@ T_ROW_COUNT       : R O W '_' C O U N T ;
 T_RR              : R R;
 T_RS              : R S ;
 T_TRIM            : T R I M ;
+T_SCHEMA          : S C H E M A ;
 T_SECURITY        : S E C U R I T Y ; 
 T_SEL             : S E L ;
 T_SELECT          : S E L E C T ; 
@@ -1302,6 +1327,7 @@ T_STEP            : S T E P ;
 T_STRING          : S T R I N G ;
 T_SUBSTRING       : S U B S T R I N G ; 
 T_SUM             : S U M ;
+T_SYS_REFCURSOR   : S Y S '_' R E F C U R S O R ; 
 T_TABLE           : T A B L E ;
 T_TEMPORARY       : T E M P O R A R Y ;
 T_TERMINATED      : T E R M I N A T E D ; 

http://git-wip-us.apache.org/repos/asf/hive/blob/92e98858/hplsql/src/main/java/org/apache/hive/hplsql/Conn.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Conn.java b/hplsql/src/main/java/org/apache/hive/hplsql/Conn.java
index ac4b521..c8cc910 100644
--- a/hplsql/src/main/java/org/apache/hive/hplsql/Conn.java
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Conn.java
@@ -57,6 +57,7 @@ public class Conn {
       Connection conn = getConnection(connName);
       runPreSql(connName, conn);
       Statement stmt = conn.createStatement();
+      exec.info(null, "Starting query");
       timer.start();
       ResultSet rs = stmt.executeQuery(query.sql);
       timer.stop();
@@ -84,10 +85,15 @@ public class Conn {
       runPreSql(connName, conn);
       Statement stmt = conn.createStatement();
       ResultSet rs = null;
+      exec.info(null, "Starting SQL statement");
+      timer.start();
       if (stmt.execute(sql)) {
         rs = stmt.getResultSet();        
       } 
       query.set(conn, stmt, rs);
+      if (info) {
+        exec.info(null, "SQL statement executed successfully (" + timer.format() + ")");
+      } 
     } catch (Exception e) {
       query.setError(e);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/92e98858/hplsql/src/main/java/org/apache/hive/hplsql/Exec.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Exec.java b/hplsql/src/main/java/org/apache/hive/hplsql/Exec.java
index b35344f..e143136 100644
--- a/hplsql/src/main/java/org/apache/hive/hplsql/Exec.java
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Exec.java
@@ -232,7 +232,7 @@ public class Exec extends HplsqlBaseVisitor<Integer> {
   /**
    * Push a boolean value to the stack
    */
-  public void stackPush(boolean val) {
+  public void stackPush(Boolean val) {
     exec.stack.push(new Var(val));  
   }
 
@@ -832,7 +832,7 @@ public class Exec extends HplsqlBaseVisitor<Integer> {
     while (!signals.empty()) {
       Signal sig = signals.pop();
       if (sig.type == Signal.Type.SQLEXCEPTION) {
-        System.err.println("Unhandled exception in PL/HQL");
+        System.err.println("Unhandled exception in HPL/SQL");
       }
       if (sig.exception != null) {
         sig.exception.printStackTrace(); 
@@ -1320,6 +1320,15 @@ public class Exec extends HplsqlBaseVisitor<Integer> {
     }
     return 0; 
   }
+  
+  /**
+   * Cursor attribute %ISOPEN, %FOUND and %NOTFOUND
+   */
+  @Override 
+  public Integer visitExpr_cursor_attribute(HplsqlParser.Expr_cursor_attributeContext ctx) {
+    exec.expr.execCursorAttribute(ctx);
+    return 0; 
+  }
     
   /**
    * Function call
@@ -1498,6 +1507,14 @@ public class Exec extends HplsqlBaseVisitor<Integer> {
     return exec.stmt.return_(ctx); 
   }  
   
+  /** 
+   * SET session options
+   */
+  @Override 
+  public Integer visitSet_current_schema_option(HplsqlParser.Set_current_schema_optionContext ctx) { 
+    return exec.stmt.setCurrentSchema(ctx); 
+  }
+  
   /**
    * MAP OBJECT statement
    */

http://git-wip-us.apache.org/repos/asf/hive/blob/92e98858/hplsql/src/main/java/org/apache/hive/hplsql/Expression.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Expression.java b/hplsql/src/main/java/org/apache/hive/hplsql/Expression.java
index f8b01e1..772c98a 100644
--- a/hplsql/src/main/java/org/apache/hive/hplsql/Expression.java
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Expression.java
@@ -268,6 +268,34 @@ public class Expression {
   }
   
   /**
+   * Cursor attribute %ISOPEN, %FOUND and %NOTFOUND
+   */
+  public void execCursorAttribute(HplsqlParser.Expr_cursor_attributeContext ctx) {
+    String name = ctx.ident().getText();
+    Var val = new Var(Var.Type.BOOL);
+    Var cursor = exec.findCursor(name);
+    if (cursor != null) {
+      Query query = (Query)cursor.value;
+      if (query != null) {
+        if (ctx.T_ISOPEN() != null) {
+          val.setValue(query.isOpen());
+        }
+        else if (ctx.T_FOUND() != null) {
+          val.setValue(query.isFound());
+        }
+        else if (ctx.T_NOTFOUND() != null) {
+          val.setValue(query.isNotFound());
+        }
+      }
+      exec.stackPush(val);
+    }
+    else {
+      trace(ctx, "Cursor not found: " + name);
+      exec.signal(Signal.Type.SQLEXCEPTION);
+    }
+  }
+  
+  /**
    * Addition operator
    */
   public void operatorAdd(HplsqlParser.ExprContext ctx) {

http://git-wip-us.apache.org/repos/asf/hive/blob/92e98858/hplsql/src/main/java/org/apache/hive/hplsql/Query.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Query.java b/hplsql/src/main/java/org/apache/hive/hplsql/Query.java
index eaaaa67..08cd6a7 100644
--- a/hplsql/src/main/java/org/apache/hive/hplsql/Query.java
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Query.java
@@ -34,6 +34,9 @@ public class Query {
   Statement stmt;
   ResultSet rs;
   Exception exception;
+
+  public enum State { OPEN, FETCHED_OK, FETCHED_NODATA, CLOSE };
+  State state = State.CLOSE;
   
   boolean withReturn = false;
   
@@ -51,6 +54,21 @@ public class Query {
     this.conn = conn;
     this.stmt = stmt;
     this.rs = rs;
+    if (rs != null) {
+      state = State.OPEN;
+    }
+  }
+  
+  /**
+   * Set the fetch status
+   */
+  public void setFetch(boolean ok) {
+    if (ok == true) {
+      state = State.FETCHED_OK;
+    }
+    else {
+      state = State.FETCHED_NODATA;
+    }
   }
   
   /**
@@ -66,6 +84,42 @@ public class Query {
   }
   
   /**
+   * Check if the cursor is open
+   */
+  public boolean isOpen() {
+    if (rs != null) {
+      return true;
+    }
+    return false;
+  }
+  
+  /**
+   * Check if the cursor was fetched and a row was returned
+   */
+  public Boolean isFound() {
+    if (state == State.OPEN || state == State.CLOSE) {
+      return null;
+    }
+    if (state == State.FETCHED_OK) {
+      return Boolean.valueOf(true);
+    } 
+    return Boolean.valueOf(false);    
+  }
+  
+  /**
+   * Check if the cursor was fetched and no row was returned
+   */
+  public Boolean isNotFound() {
+    if (state == State.OPEN || state == State.CLOSE) {
+      return null;
+    }
+    if (state == State.FETCHED_NODATA) {
+      return Boolean.valueOf(true);
+    }
+    return Boolean.valueOf(false);
+  }
+  
+  /**
    * Close statement results
    */
   public void closeStatement() {
@@ -78,6 +132,7 @@ public class Query {
         stmt.close();
         stmt = null;
       }
+      state = State.CLOSE;
     } catch (SQLException e) {
       e.printStackTrace();
     }   

http://git-wip-us.apache.org/repos/asf/hive/blob/92e98858/hplsql/src/main/java/org/apache/hive/hplsql/Stmt.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Stmt.java b/hplsql/src/main/java/org/apache/hive/hplsql/Stmt.java
index bfb76cd..9a5e970 100644
--- a/hplsql/src/main/java/org/apache/hive/hplsql/Stmt.java
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Stmt.java
@@ -280,18 +280,24 @@ public class Stmt {
     Var var = null;
     String cursor = ctx.L_ID().toString();   
     String sql = null;
-    // Dynamic SQL
-    if (ctx.T_FOR() != null) {
-      sql = evalPop(ctx.expr()).toString();
-      if (trace) {
-        trace(ctx, cursor + ": " + sql);
+    if (ctx.T_FOR() != null) {                             // SELECT statement or dynamic SQL
+      if (ctx.expr() != null) {
+        sql = evalPop(ctx.expr()).toString();
+      }
+      else {
+        sql = evalPop(ctx.select_stmt()).toString();
       }
       query = new Query(sql);
-      var = new Var(cursor, Type.CURSOR, query);
-      exec.addVariable(var);
+      var = exec.findCursor(cursor);                      // Can be a ref cursor variable
+      if (var == null) {
+        var = new Var(cursor, Type.CURSOR, query);
+        exec.addVariable(var);
+      }
+      else {
+        var.setValue(query);
+      }
     }
-    // Declared cursor
-    else {
+    else {                                                 // Declared cursor
       var = exec.findVariable(cursor);      
       if (var != null && var.type == Type.CURSOR) {
         query = (Query)var.value;
@@ -303,13 +309,12 @@ public class Stmt {
           sql = evalPop(query.sqlSelect).toString();
           query.setSql(sql);
         }
-        if (trace) {
-          trace(ctx, cursor + ": " + sql);
-        } 
       }
     }
-    // Open cursor now
     if (query != null) {
+      if (trace) {
+        trace(ctx, cursor + ": " + sql);
+      } 
       exec.executeQuery(ctx, query, exec.conf.defaultConnection);
       if (query.error()) {
         exec.signal(query);
@@ -343,7 +348,13 @@ public class Stmt {
       exec.setSqlCode(-1);
       exec.signal(Signal.Type.SQLEXCEPTION);
       return 1;
-    }    
+    }  
+    else if (cursor.value == null) {
+      trace(ctx, "Cursor not open: " + name);
+      exec.setSqlCode(-1);
+      exec.signal(Signal.Type.SQLEXCEPTION);
+      return 1;
+    }  
     else if (exec.getOffline()) {
       exec.setSqlCode(100);
       exec.signal(Signal.Type.NOTFOUND);
@@ -360,6 +371,7 @@ public class Stmt {
       if(rs != null && rsm != null) {
         int cols = ctx.L_ID().size() - 1;
         if(rs.next()) {
+          query.setFetch(true);
           for(int i=1; i <= cols; i++) {
             Var var = exec.findVariable(ctx.L_ID(i).getText());
             if(var != null) {
@@ -377,6 +389,7 @@ public class Stmt {
           exec.setSqlSuccess();
         }
         else {
+          query.setFetch(false);
           exec.setSqlCode(100);
         }
       }
@@ -639,12 +652,13 @@ public class Stmt {
    * USE statement
    */
   public Integer use(HplsqlParser.Use_stmtContext ctx) {
+    trace(ctx, "USE");
+    return use(ctx, ctx.T_USE().toString() + " " + evalPop(ctx.expr()).toString());
+  }
+  
+  public Integer use(ParserRuleContext ctx, String sql) {
     if(trace) {
-      trace(ctx, "USE");
-    }
-    String sql = ctx.T_USE().toString() + " " + evalPop(ctx.expr()).toString();
-    if(trace) {
-      trace(ctx, "Query: " + sql);
+      trace(ctx, "SQL statement: " + sql);
     }    
     Query query = exec.executeSql(ctx, sql, exec.conf.defaultConnection);
     if(query.error()) {
@@ -959,6 +973,14 @@ public class Stmt {
 	  return 0; 
   }
   
+  /** 
+   * SET current schema 
+   */
+  public Integer setCurrentSchema(HplsqlParser.Set_current_schema_optionContext ctx) { 
+    trace(ctx, "SET CURRENT SCHEMA");
+    return use(ctx, "USE " + evalPop(ctx.expr()).toString());
+  }
+  
   /**
    * SIGNAL statement
    */

http://git-wip-us.apache.org/repos/asf/hive/blob/92e98858/hplsql/src/main/java/org/apache/hive/hplsql/Var.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Var.java b/hplsql/src/main/java/org/apache/hive/hplsql/Var.java
index 87b42f9..7e4ae52 100644
--- a/hplsql/src/main/java/org/apache/hive/hplsql/Var.java
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Var.java
@@ -32,7 +32,7 @@ import java.sql.Timestamp;
 public class Var {
 
 	// Data types
-	public enum Type {BOOL, CURSOR, DATE, DEC, FILE, IDENT, BIGINT, INTERVAL, RS_LOCATOR, STRING, STRINGLIST, TIMESTAMP, NULL};
+	public enum Type {BOOL, CURSOR, DATE, DECIMAL, FILE, IDENT, BIGINT, INTERVAL, RS_LOCATOR, STRING, STRINGLIST, TIMESTAMP, NULL};
 	public static Var Empty = new Var();
 	public static Var Null = new Var(Type.NULL);
 	
@@ -61,7 +61,7 @@ public class Var {
 	}
 	
 	public Var(BigDecimal value) {
-    this.type = Type.DEC;
+    this.type = Type.DECIMAL;
     this.value = value;
   }
   
@@ -194,6 +194,13 @@ public class Var {
     return this;
   }
 	
+	public Var setValue(Boolean val) {
+    if (type == Type.BOOL) {
+      value = val;
+    }
+    return this;
+  }
+	
 	public void setValue(Object value) {
     this.value = value;
   }
@@ -209,6 +216,9 @@ public class Var {
     else if (type == java.sql.Types.INTEGER || type == java.sql.Types.BIGINT) {
       cast(new Var(new Long(rs.getLong(idx))));
     }
+    else if (type == java.sql.Types.DECIMAL || type == java.sql.Types.NUMERIC) {
+      cast(new Var(rs.getBigDecimal(idx)));
+    }
     return this;
   }
 	
@@ -239,12 +249,18 @@ public class Var {
     else if (type.equalsIgnoreCase("CHAR") || type.equalsIgnoreCase("VARCHAR") || type.equalsIgnoreCase("STRING")) {
       return Type.STRING;
     }
+    else if (type.equalsIgnoreCase("DEC") || type.equalsIgnoreCase("DECIMAL") || type.equalsIgnoreCase("NUMERIC")) {
+      return Type.DECIMAL;
+    }
     else if (type.equalsIgnoreCase("DATE")) {
       return Type.DATE;
     }
     else if (type.equalsIgnoreCase("TIMESTAMP")) {
       return Type.TIMESTAMP;
     }
+    else if (type.equalsIgnoreCase("SYS_REFCURSOR")) {
+      return Type.CURSOR;
+    }
     else if (type.equalsIgnoreCase("UTL_FILE.FILE_TYPE")) {
       return Type.FILE;
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/92e98858/hplsql/src/test/queries/db/cursor_attributes.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/db/cursor_attributes.sql b/hplsql/src/test/queries/db/cursor_attributes.sql
new file mode 100644
index 0000000..9b1e9db
--- /dev/null
+++ b/hplsql/src/test/queries/db/cursor_attributes.sql
@@ -0,0 +1,60 @@
+DECLARE 
+  CURSOR c1 IS SELECT 'A' FROM src LIMIT 1;
+  v1 VARCHAR(30);
+BEGIN
+  IF c1%ISOPEN THEN
+    DBMS_OUTPUT.PUT_LINE('%ISOPEN failed');
+  ELSE 
+    DBMS_OUTPUT.PUT_LINE('%ISOPEN correct');
+  END IF; 
+  
+  OPEN c1;
+
+  IF c1%ISOPEN THEN
+    DBMS_OUTPUT.PUT_LINE('%ISOPEN correct');
+  ELSE 
+    DBMS_OUTPUT.PUT_LINE('%ISOPEN failed');
+  END IF; 
+  
+  IF c1%FOUND IS NULL THEN
+    DBMS_OUTPUT.PUT_LINE('%FOUND correct');
+  ELSE
+    DBMS_OUTPUT.PUT_LINE('%FOUND failed');
+  END IF;
+
+  IF c1%NOTFOUND IS NULL THEN
+    DBMS_OUTPUT.PUT_LINE('%NOTFOUND correct');
+  ELSE
+    DBMS_OUTPUT.PUT_LINE('%NOTFOUND failed');
+  END IF;
+  
+  FETCH c1 INTO v1;
+  
+  IF c1%FOUND THEN
+    DBMS_OUTPUT.PUT_LINE('%FOUND correct');
+  ELSE
+    DBMS_OUTPUT.PUT_LINE('%FOUND failed');
+  END IF;
+
+  IF c1%NOTFOUND THEN
+    DBMS_OUTPUT.PUT_LINE('%NOTFOUND failed');
+  ELSE
+    DBMS_OUTPUT.PUT_LINE('%NOTFOUND correct');
+  END IF;  
+  
+  FETCH c1 INTO v1;
+  
+  IF c1%FOUND THEN
+    DBMS_OUTPUT.PUT_LINE('%FOUND failed');
+  ELSE
+    DBMS_OUTPUT.PUT_LINE('%FOUND correct');
+  END IF;
+
+  IF c1%NOTFOUND THEN
+    DBMS_OUTPUT.PUT_LINE('%NOTFOUND correct');
+  ELSE
+    DBMS_OUTPUT.PUT_LINE('%NOTFOUND failed');
+  END IF;  
+
+  CLOSE c1; 
+END;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/92e98858/hplsql/src/test/queries/db/select_into.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/db/select_into.sql b/hplsql/src/test/queries/db/select_into.sql
new file mode 100644
index 0000000..3995ba2
--- /dev/null
+++ b/hplsql/src/test/queries/db/select_into.sql
@@ -0,0 +1,17 @@
+DECLARE v_int INT;
+DECLARE v_dec DECIMAL(18,2);
+DECLARE v_dec0 DECIMAL(18,0);
+
+SELECT TOP 1 
+  CAST(1 AS INT), 
+  CAST(1.1 AS DECIMAL(18,2)),
+  CAST(1.1 AS DECIMAL(18,0))   
+INTO 
+  v_int,
+  v_dec,
+  v_dec0  
+FROM src ;
+        
+PRINT 'INT: ' || v_int;
+PRINT 'DECIMAL: ' || v_dec;
+PRINT 'DECIMAL0: ' || v_dec0;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/92e98858/hplsql/src/test/queries/db/set_current_schema.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/db/set_current_schema.sql b/hplsql/src/test/queries/db/set_current_schema.sql
new file mode 100644
index 0000000..059ef22
--- /dev/null
+++ b/hplsql/src/test/queries/db/set_current_schema.sql
@@ -0,0 +1,6 @@
+SET CURRENT SCHEMA = default;
+SET CURRENT_SCHEMA = default;
+SET SCHEMA = default;
+SET SCHEMA default;
+SET SCHEMA = 'default';
+SET SCHEMA 'def' || 'ault';
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/92e98858/hplsql/src/test/queries/db/sys_refcursor.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/db/sys_refcursor.sql b/hplsql/src/test/queries/db/sys_refcursor.sql
new file mode 100644
index 0000000..eeb98ed
--- /dev/null
+++ b/hplsql/src/test/queries/db/sys_refcursor.sql
@@ -0,0 +1,65 @@
+CREATE PROCEDURE spResultSet1 (cur OUT SYS_REFCURSOR)
+BEGIN
+  OPEN cur FOR SELECT 'A' FROM src LIMIT 1;
+END; 
+
+DECLARE 
+  c1 SYS_REFCURSOR;
+  v1 VARCHAR(30);
+BEGIN
+  IF c1%ISOPEN THEN
+    DBMS_OUTPUT.PUT_LINE('%ISOPEN failed');
+  ELSE 
+    DBMS_OUTPUT.PUT_LINE('%ISOPEN correct');
+  END IF; 
+  
+  CALL spResultSet1(c1);
+  
+  IF c1%ISOPEN THEN
+    DBMS_OUTPUT.PUT_LINE('%ISOPEN correct');
+  ELSE 
+    DBMS_OUTPUT.PUT_LINE('%ISOPEN failed');
+  END IF; 
+  
+  IF c1%FOUND IS NULL THEN
+    DBMS_OUTPUT.PUT_LINE('%FOUND correct');
+  ELSE
+    DBMS_OUTPUT.PUT_LINE('%FOUND failed');
+  END IF;
+
+  IF c1%NOTFOUND IS NULL THEN
+    DBMS_OUTPUT.PUT_LINE('%NOTFOUND correct');
+  ELSE
+    DBMS_OUTPUT.PUT_LINE('%NOTFOUND failed');
+  END IF;
+  
+  FETCH c1 INTO v1;
+  
+  IF c1%FOUND THEN
+    DBMS_OUTPUT.PUT_LINE('%FOUND correct');
+  ELSE
+    DBMS_OUTPUT.PUT_LINE('%FOUND failed');
+  END IF;
+
+  IF c1%NOTFOUND THEN
+    DBMS_OUTPUT.PUT_LINE('%NOTFOUND failed');
+  ELSE
+    DBMS_OUTPUT.PUT_LINE('%NOTFOUND correct');
+  END IF;  
+  
+  FETCH c1 INTO v1;
+  
+  IF c1%FOUND THEN
+    DBMS_OUTPUT.PUT_LINE('%FOUND failed');
+  ELSE
+    DBMS_OUTPUT.PUT_LINE('%FOUND correct');
+  END IF;
+
+  IF c1%NOTFOUND THEN
+    DBMS_OUTPUT.PUT_LINE('%NOTFOUND correct');
+  ELSE
+    DBMS_OUTPUT.PUT_LINE('%NOTFOUND failed');
+  END IF;  
+
+  CLOSE c1; 
+END;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/92e98858/hplsql/src/test/queries/db/use.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/db/use.sql b/hplsql/src/test/queries/db/use.sql
new file mode 100644
index 0000000..dab0615
--- /dev/null
+++ b/hplsql/src/test/queries/db/use.sql
@@ -0,0 +1,2 @@
+USE default;
+USE SUBSTR('default', 1, 7);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/92e98858/hplsql/src/test/results/db/cursor_attributes.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/db/cursor_attributes.out.txt b/hplsql/src/test/results/db/cursor_attributes.out.txt
new file mode 100644
index 0000000..fbc8eef
--- /dev/null
+++ b/hplsql/src/test/results/db/cursor_attributes.out.txt
@@ -0,0 +1,33 @@
+Ln:2 DECLARE CURSOR c1
+Ln:3 DECLARE v1 VARCHAR
+Ln:5 IF
+Ln:5 ELSE executed
+%ISOPEN correct
+Ln:11 OPEN
+Ln:11 c1: SELECT 'A' FROM src LIMIT 1
+Ln:13 IF
+Ln:13 IF TRUE executed
+%ISOPEN correct
+Ln:19 IF
+Ln:19 IF TRUE executed
+%FOUND correct
+Ln:25 IF
+Ln:25 IF TRUE executed
+%NOTFOUND correct
+Ln:31 FETCH
+Ln:31 COLUMN: _c0, string
+Ln:31 SET v1 = A
+Ln:33 IF
+Ln:33 IF TRUE executed
+%FOUND correct
+Ln:39 IF
+Ln:39 ELSE executed
+%NOTFOUND correct
+Ln:45 FETCH
+Ln:47 IF
+Ln:47 ELSE executed
+%FOUND correct
+Ln:53 IF
+Ln:53 IF TRUE executed
+%NOTFOUND correct
+Ln:59 CLOSE
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/92e98858/hplsql/src/test/results/db/select_into.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/db/select_into.out.txt b/hplsql/src/test/results/db/select_into.out.txt
new file mode 100644
index 0000000..80d067e
--- /dev/null
+++ b/hplsql/src/test/results/db/select_into.out.txt
@@ -0,0 +1,19 @@
+Ln:1 DECLARE v_int INT
+Ln:2 DECLARE v_dec DECIMAL
+Ln:3 DECLARE v_dec0 DECIMAL
+Ln:5 SELECT
+Ln:5 SELECT CAST(1 AS INT), CAST(1.1 AS DECIMAL(18,2)), CAST(1.1 AS DECIMAL(18,0)) FROM src LIMIT 1
+Ln:5 SELECT completed successfully
+Ln:5 SELECT INTO statement executed
+Ln:5 COLUMN: _c0, int
+Ln:5 SET v_int = 1
+Ln:5 COLUMN: _c1, decimal
+Ln:5 SET v_dec = 1.1
+Ln:5 COLUMN: _c2, decimal
+Ln:5 SET v_dec0 = 1
+Ln:15 PRINT
+INT: 1
+Ln:16 PRINT
+DECIMAL: 1.1
+Ln:17 PRINT
+DECIMAL0: 1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/92e98858/hplsql/src/test/results/db/set_current_schema.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/db/set_current_schema.out.txt b/hplsql/src/test/results/db/set_current_schema.out.txt
new file mode 100644
index 0000000..6ef8572
--- /dev/null
+++ b/hplsql/src/test/results/db/set_current_schema.out.txt
@@ -0,0 +1,12 @@
+Ln:1 SET CURRENT SCHEMA
+Ln:1 SQL statement: USE default
+Ln:2 SET CURRENT SCHEMA
+Ln:2 SQL statement: USE default
+Ln:3 SET CURRENT SCHEMA
+Ln:3 SQL statement: USE default
+Ln:4 SET CURRENT SCHEMA
+Ln:4 SQL statement: USE default
+Ln:5 SET CURRENT SCHEMA
+Ln:5 SQL statement: USE default
+Ln:6 SET CURRENT SCHEMA
+Ln:6 SQL statement: USE default
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/92e98858/hplsql/src/test/results/db/sys_refcursor.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/db/sys_refcursor.out.txt b/hplsql/src/test/results/db/sys_refcursor.out.txt
new file mode 100644
index 0000000..08239cf
--- /dev/null
+++ b/hplsql/src/test/results/db/sys_refcursor.out.txt
@@ -0,0 +1,36 @@
+Ln:1 CREATE PROCEDURE spResultSet1
+Ln:7 DECLARE c1 SYS_REFCURSOR
+Ln:8 DECLARE v1 VARCHAR
+Ln:10 IF
+Ln:10 ELSE executed
+%ISOPEN correct
+Ln:16 EXEC PROCEDURE spResultSet1
+Ln:16 SET PARAM cur = null
+Ln:3 OPEN
+Ln:3 cur: SELECT 'A' FROM src LIMIT 1
+Ln:18 IF
+Ln:18 IF TRUE executed
+%ISOPEN correct
+Ln:24 IF
+Ln:24 IF TRUE executed
+%FOUND correct
+Ln:30 IF
+Ln:30 IF TRUE executed
+%NOTFOUND correct
+Ln:36 FETCH
+Ln:36 COLUMN: _c0, string
+Ln:36 SET v1 = A
+Ln:38 IF
+Ln:38 IF TRUE executed
+%FOUND correct
+Ln:44 IF
+Ln:44 ELSE executed
+%NOTFOUND correct
+Ln:50 FETCH
+Ln:52 IF
+Ln:52 ELSE executed
+%FOUND correct
+Ln:58 IF
+Ln:58 IF TRUE executed
+%NOTFOUND correct
+Ln:64 CLOSE
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/92e98858/hplsql/src/test/results/db/use.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/db/use.out.txt b/hplsql/src/test/results/db/use.out.txt
new file mode 100644
index 0000000..4a478f7
--- /dev/null
+++ b/hplsql/src/test/results/db/use.out.txt
@@ -0,0 +1,4 @@
+Ln:1 USE
+Ln:1 SQL statement: USE default
+Ln:2 USE
+Ln:2 SQL statement: USE default
\ No newline at end of file


[11/43] hive git commit: HIVE-11303: Getting Tez LimitExceededException after dag execution on large query (Jason Dere, reviewed by Gopal V)

Posted by xu...@apache.org.
HIVE-11303: Getting Tez LimitExceededException after dag execution on large query (Jason Dere, reviewed by Gopal V)


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

Branch: refs/heads/spark
Commit: 72f97fc7760134465333983fc40766e9e864e643
Parents: bab3ee3
Author: Jason Dere <jd...@hortonworks.com>
Authored: Tue Jul 21 17:12:35 2015 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Tue Jul 21 17:12:35 2015 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java  | 10 ++++++++--
 1 file changed, 8 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/72f97fc7/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
index 2ea83ab..73263ee 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezTask.java
@@ -171,8 +171,14 @@ public class TezTask extends Task<TezWork> {
       }
 
       // fetch the counters
-      Set<StatusGetOpts> statusGetOpts = EnumSet.of(StatusGetOpts.GET_COUNTERS);
-      counters = client.getDAGStatus(statusGetOpts).getDAGCounters();
+      try {
+        Set<StatusGetOpts> statusGetOpts = EnumSet.of(StatusGetOpts.GET_COUNTERS);
+        counters = client.getDAGStatus(statusGetOpts).getDAGCounters();
+      } catch (Exception err) {
+        // Don't fail execution due to counters - just don't print summary info
+        LOG.error("Failed to get counters: " + err, err);
+        counters = null;
+      }
       TezSessionPoolManager.getInstance().returnSession(session);
 
       if (LOG.isInfoEnabled() && counters != null


[06/43] hive git commit: HIVE-11320 ACID enable predicate pushdown for insert-only delta file (Eugene Koifman, reviewed by Alan Gates)

Posted by xu...@apache.org.
HIVE-11320 ACID enable predicate pushdown for insert-only delta file (Eugene Koifman, reviewed by Alan Gates)


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

Branch: refs/heads/spark
Commit: 990416249833e722ca8a32dd9dd425883da0caaf
Parents: 6ec72de
Author: Eugene Koifman <ek...@hortonworks.com>
Authored: Tue Jul 21 11:42:14 2015 -0700
Committer: Eugene Koifman <ek...@hortonworks.com>
Committed: Tue Jul 21 11:42:14 2015 -0700

----------------------------------------------------------------------
 .../hive/ql/io/orc/OrcRawRecordMerger.java      | 20 ++++--
 .../apache/hadoop/hive/ql/TestTxnCommands2.java | 68 ++++++++++++++++++--
 2 files changed, 75 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/99041624/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java
index 2f11611..58b85ef 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRawRecordMerger.java
@@ -478,10 +478,6 @@ public class OrcRawRecordMerger implements AcidInputFormat.RawReader<OrcStruct>{
 
     // we always want to read all of the deltas
     eventOptions.range(0, Long.MAX_VALUE);
-    // Turn off the sarg before pushing it to delta.  We never want to push a sarg to a delta as
-    // it can produce wrong results (if the latest valid version of the record is filtered out by
-    // the sarg) or ArrayOutOfBounds errors (when the sarg is applied to a delete record)
-    eventOptions.searchArgument(null, null);
     if (deltaDirectory != null) {
       for(Path delta: deltaDirectory) {
         ReaderKey key = new ReaderKey();
@@ -492,8 +488,20 @@ public class OrcRawRecordMerger implements AcidInputFormat.RawReader<OrcStruct>{
         if (length != -1 && fs.exists(deltaFile)) {
           Reader deltaReader = OrcFile.createReader(deltaFile,
               OrcFile.readerOptions(conf).maxLength(length));
-          ReaderPair deltaPair = new ReaderPair(key, deltaReader, bucket, minKey,
-            maxKey, eventOptions, deltaDir.getStatementId());
+          Reader.Options deltaEventOptions = null;
+          if(eventOptions.getSearchArgument() != null) {
+            // Turn off the sarg before pushing it to delta.  We never want to push a sarg to a delta as
+            // it can produce wrong results (if the latest valid version of the record is filtered out by
+            // the sarg) or ArrayOutOfBounds errors (when the sarg is applied to a delete record)
+            // unless the delta only has insert events
+            OrcRecordUpdater.AcidStats acidStats = OrcRecordUpdater.parseAcidStats(deltaReader);
+            if(acidStats.deletes > 0 || acidStats.updates > 0) {
+              deltaEventOptions = eventOptions.clone().searchArgument(null, null);
+            }
+          }
+          ReaderPair deltaPair;
+          deltaPair = new ReaderPair(key, deltaReader, bucket, minKey,
+            maxKey, deltaEventOptions != null ? deltaEventOptions : eventOptions, deltaDir.getStatementId());
           if (deltaPair.nextRecord != null) {
             readers.put(key, deltaPair);
           }

http://git-wip-us.apache.org/repos/asf/hive/blob/99041624/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java
index 33ca998..57e4fb9 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java
@@ -20,13 +20,11 @@ package org.apache.hadoop.hive.ql;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.txn.TxnDbUtil;
-import org.apache.hadoop.hive.ql.io.AcidUtils;
-import org.apache.hadoop.hive.ql.io.orc.FileDump;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.hive.ql.txn.compactor.Worker;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -36,13 +34,11 @@ import org.junit.Test;
 import org.junit.rules.TestName;
 
 import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.FileOutputStream;
-import java.io.FilenameFilter;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Comparator;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 /**
  * TODO: this should be merged with TestTxnCommands once that is checked in
@@ -55,7 +51,7 @@ public class TestTxnCommands2 {
   ).getPath().replaceAll("\\\\", "/");
   private static final String TEST_WAREHOUSE_DIR = TEST_DATA_DIR + "/warehouse";
   //bucket count for test tables; set it to 1 for easier debugging
-  private static int BUCKET_COUNT = 2;
+  private static int BUCKET_COUNT = 1;
   @Rule
   public TestName testName = new TestName();
   private HiveConf hiveConf;
@@ -122,6 +118,64 @@ public class TestTxnCommands2 {
       FileUtils.deleteDirectory(new File(TEST_DATA_DIR));
     }
   }
+  @Test
+  public void testOrcPPD() throws Exception  {
+    testOrcPPD(true);
+  }
+  @Test
+  public void testOrcNoPPD() throws Exception {
+    testOrcPPD(false);
+  }
+  private void testOrcPPD(boolean enablePPD) throws Exception  {
+    boolean originalPpd = hiveConf.getBoolVar(HiveConf.ConfVars.HIVEOPTINDEXFILTER);
+    hiveConf.setBoolVar(HiveConf.ConfVars.HIVEOPTINDEXFILTER, enablePPD);//enables ORC PPD
+    int[][] tableData = {{1,2},{3,4}};
+    runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) " + makeValuesClause(tableData));
+    runStatementOnDriver("alter table "+ Table.ACIDTBL + " compact 'MAJOR'");
+    Worker t = new Worker();
+    t.setThreadId((int) t.getId());
+    t.setHiveConf(hiveConf);
+    AtomicBoolean stop = new AtomicBoolean();
+    AtomicBoolean looped = new AtomicBoolean();
+    stop.set(true);
+    t.init(stop, looped);
+    t.run();
+    //now we have base_0001 file
+    int[][] tableData2 = {{1,7},{5,6},{7,8},{9,10}};
+    runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) " + makeValuesClause(tableData2));
+    //now we have delta_0002_0002_0000 with inserts only (ok to push predicate)
+    runStatementOnDriver("delete from " + Table.ACIDTBL + " where a=7 and b=8");
+    //now we have delta_0003_0003_0000 with delete events (can't push predicate)
+    runStatementOnDriver("update " + Table.ACIDTBL + " set b = 11 where a = 9");
+    //and another delta with update op
+    List<String> rs1 = runStatementOnDriver("select a,b from " + Table.ACIDTBL + " where a > 1 order by a,b");
+    int [][] resultData = {{3,4},{5,6},{9,11}};
+    Assert.assertEquals("Update failed", stringifyValues(resultData), rs1);
+    hiveConf.setBoolVar(HiveConf.ConfVars.HIVEOPTINDEXFILTER, originalPpd);
+  }
+  @Ignore("alter table")
+  @Test
+  public void testAlterTable() throws Exception {
+    int[][] tableData = {{1,2}};
+    runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) " + makeValuesClause(tableData));
+    runStatementOnDriver("alter table "+ Table.ACIDTBL + " compact 'MAJOR'");
+    Worker t = new Worker();
+    t.setThreadId((int) t.getId());
+    t.setHiveConf(hiveConf);
+    AtomicBoolean stop = new AtomicBoolean();
+    AtomicBoolean looped = new AtomicBoolean();
+    stop.set(true);
+    t.init(stop, looped);
+    t.run();
+    int[][] tableData2 = {{5,6}};
+    runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) " + makeValuesClause(tableData2));
+    List<String> rs1 = runStatementOnDriver("select a,b from " + Table.ACIDTBL + " where b > 0 order by a,b");
+
+    runStatementOnDriver("alter table " + Table.ACIDTBL + " add columns(c int)");
+    int[][] moreTableData = {{7,8,9}};
+    runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b,c) " + makeValuesClause(moreTableData));
+    List<String> rs0 = runStatementOnDriver("select a,b,c from " + Table.ACIDTBL + " where a > 0 order by a,b,c");
+  }
   @Ignore("not needed but useful for testing")
   @Test
   public void testNonAcidInsert() throws Exception {


[24/43] hive git commit: HIVE-11347: CBO: Calcite Operator To Hive Operator (Calcite Return Path): fix CTAS (Pengcheng Xiong, via Jesus Camacho Rodriguez)

Posted by xu...@apache.org.
HIVE-11347: CBO: Calcite Operator To Hive Operator (Calcite Return Path): fix CTAS (Pengcheng Xiong, via 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/65396f04
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/65396f04
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/65396f04

Branch: refs/heads/spark
Commit: 65396f04fec331bd6ec8227e864f2600f0f7e925
Parents: 8dcead4
Author: Pengcheng Xiong <px...@hortonworks.com>
Authored: Tue Jul 28 09:03:54 2015 +0100
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Tue Jul 28 09:03:54 2015 +0100

----------------------------------------------------------------------
 .../optimizer/calcite/translator/PlanModifierForASTConv.java   | 2 +-
 .../calcite/translator/PlanModifierForReturnPath.java          | 6 +++++-
 .../java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java   | 3 ++-
 3 files changed, 8 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/65396f04/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java
index 5f6be9e..5cd3a06 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java
@@ -169,7 +169,7 @@ public class PlanModifierForASTConv {
     }
   }
 
-  private static RelNode renameTopLevelSelectInResultSchema(final RelNode rootRel,
+  public static RelNode renameTopLevelSelectInResultSchema(final RelNode rootRel,
       Pair<RelNode, RelNode> topSelparentPair, List<FieldSchema> resultSchema)
       throws CalciteSemanticException {
     RelNode parentOforiginalProjRel = topSelparentPair.getKey();

http://git-wip-us.apache.org/repos/asf/hive/blob/65396f04/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForReturnPath.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForReturnPath.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForReturnPath.java
index 06cf69d..81cc474 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForReturnPath.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForReturnPath.java
@@ -28,12 +28,16 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil;
 public class PlanModifierForReturnPath {
 
 
-  public static RelNode convertOpTree(RelNode rel, List<FieldSchema> resultSchema)
+  public static RelNode convertOpTree(RelNode rel, List<FieldSchema> resultSchema, boolean isCTAS)
           throws CalciteSemanticException {
     RelNode newTopNode = rel;
 
     Pair<RelNode, RelNode> topSelparentPair = HiveCalciteUtil.getTopLevelSelect(newTopNode);
     PlanModifierUtil.fixTopOBSchema(newTopNode, topSelparentPair, resultSchema, false);
+    if (isCTAS) {
+      newTopNode = PlanModifierForASTConv.renameTopLevelSelectInResultSchema(newTopNode,
+          topSelparentPair, resultSchema);
+    }
 
     return newTopNode;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/65396f04/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
----------------------------------------------------------------------
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 5b469e3..5cbc428 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
@@ -641,7 +641,8 @@ public class CalcitePlanner extends SemanticAnalyzer {
     }
 
     RelNode modifiedOptimizedOptiqPlan = PlanModifierForReturnPath.convertOpTree(
-            introduceProjectIfNeeded(optimizedOptiqPlan), topLevelFieldSchema);
+        introduceProjectIfNeeded(optimizedOptiqPlan), topLevelFieldSchema, this.getQB()
+            .getTableDesc() != null);
 
     LOG.debug("Translating the following plan:\n" + RelOptUtil.toString(modifiedOptimizedOptiqPlan));
     Operator<?> hiveRoot = new HiveOpConverter(this, conf, unparseTranslator, topOps,


[36/43] hive git commit: HIVE-11253. Move SearchArgument and VectorizedRowBatch classes to storage-api. (omalley reviewed by prasanthj)

Posted by xu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/9ae70cb4/serde/src/java/org/apache/hadoop/hive/ql/io/sarg/PredicateLeaf.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/ql/io/sarg/PredicateLeaf.java b/serde/src/java/org/apache/hadoop/hive/ql/io/sarg/PredicateLeaf.java
deleted file mode 100644
index 3a92565..0000000
--- a/serde/src/java/org/apache/hadoop/hive/ql/io/sarg/PredicateLeaf.java
+++ /dev/null
@@ -1,104 +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.
- */
-
-package org.apache.hadoop.hive.ql.io.sarg;
-
-import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
-
-import java.sql.Date;
-import java.sql.Timestamp;
-import java.util.List;
-
-/**
- * The primitive predicates that form a SearchArgument.
- */
-public interface PredicateLeaf {
-
-  /**
-   * The possible operators for predicates. To get the opposites, construct
-   * an expression with a not operator.
-   */
-  public static enum Operator {
-    EQUALS,
-    NULL_SAFE_EQUALS,
-    LESS_THAN,
-    LESS_THAN_EQUALS,
-    IN,
-    BETWEEN,
-    IS_NULL
-  }
-
-  /**
-   * The possible types for sargs.
-   */
-  public static enum Type {
-    INTEGER(Integer.class), // all of the integer types except long
-    LONG(Long.class),
-    FLOAT(Double.class),   // float and double
-    STRING(String.class),  // string, char, varchar
-    DATE(Date.class),
-    DECIMAL(HiveDecimalWritable.class),
-    TIMESTAMP(Timestamp.class),
-    BOOLEAN(Boolean.class);
-
-    private final Class cls;
-    Type(Class cls) {
-      this.cls = cls;
-    }
-
-    /**
-     * For all SARG leaves, the values must be the matching class.
-     * @return the value class
-     */
-    public Class getValueClass() {
-      return cls;
-    }
-  }
-
-  /**
-   * Get the operator for the leaf.
-   */
-  public Operator getOperator();
-
-  /**
-   * Get the type of the column and literal by the file format.
-   */
-  public Type getType();
-
-  /**
-   * Get the simple column name.
-   * @return the column name
-   */
-  public String getColumnName();
-
-  /**
-   * Get the literal half of the predicate leaf. Adapt the original type for what orc needs
-   *
-   * @return an Integer, Long, Double, or String
-   */
-  public Object getLiteral();
-
-  /**
-   * For operators with multiple literals (IN and BETWEEN), get the literals.
-   *
-   * @return the list of literals (Integer, Longs, Doubles, or Strings)
-   *
-   */
-  public List<Object> getLiteralList();
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/9ae70cb4/serde/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgument.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgument.java b/serde/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgument.java
deleted file mode 100644
index bc0d503..0000000
--- a/serde/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgument.java
+++ /dev/null
@@ -1,298 +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.
- */
-
-package org.apache.hadoop.hive.ql.io.sarg;
-
-import java.util.List;
-
-/**
- * Primary interface for <a href="http://en.wikipedia.org/wiki/Sargable">
- *   SearchArgument</a>, which are the subset of predicates
- * that can be pushed down to the RecordReader. Each SearchArgument consists
- * of a series of SearchClauses that must each be true for the row to be
- * accepted by the filter.
- *
- * This requires that the filter be normalized into conjunctive normal form
- * (<a href="http://en.wikipedia.org/wiki/Conjunctive_normal_form">CNF</a>).
- */
-public interface SearchArgument {
-
-  /**
-   * The potential result sets of logical operations.
-   */
-  public static enum TruthValue {
-    YES, NO, NULL, YES_NULL, NO_NULL, YES_NO, YES_NO_NULL;
-
-    /**
-     * Compute logical or between the two values.
-     * @param right the other argument or null
-     * @return the result
-     */
-    public TruthValue or(TruthValue right) {
-      if (right == null || right == this) {
-        return this;
-      }
-      if (right == YES || this == YES) {
-        return YES;
-      }
-      if (right == YES_NULL || this == YES_NULL) {
-        return YES_NULL;
-      }
-      if (right == NO) {
-        return this;
-      }
-      if (this == NO) {
-        return right;
-      }
-      if (this == NULL) {
-        if (right == NO_NULL) {
-          return NULL;
-        } else {
-          return YES_NULL;
-        }
-      }
-      if (right == NULL) {
-        if (this == NO_NULL) {
-          return NULL;
-        } else {
-          return YES_NULL;
-        }
-      }
-      return YES_NO_NULL;
-    }
-
-    /**
-     * Compute logical AND between the two values.
-     * @param right the other argument or null
-     * @return the result
-     */
-    public TruthValue and(TruthValue right) {
-      if (right == null || right == this) {
-        return this;
-      }
-      if (right == NO || this == NO) {
-        return NO;
-      }
-      if (right == NO_NULL || this == NO_NULL) {
-        return NO_NULL;
-      }
-      if (right == YES) {
-        return this;
-      }
-      if (this == YES) {
-        return right;
-      }
-      if (this == NULL) {
-        if (right == YES_NULL) {
-          return NULL;
-        } else {
-          return NO_NULL;
-        }
-      }
-      if (right == NULL) {
-        if (this == YES_NULL) {
-          return NULL;
-        } else {
-          return NO_NULL;
-        }
-      }
-      return YES_NO_NULL;
-    }
-
-    public TruthValue not() {
-      switch (this) {
-        case NO:
-          return YES;
-        case YES:
-          return NO;
-        case NULL:
-        case YES_NO:
-        case YES_NO_NULL:
-          return this;
-        case NO_NULL:
-          return YES_NULL;
-        case YES_NULL:
-          return NO_NULL;
-        default:
-          throw new IllegalArgumentException("Unknown value: " + this);
-      }
-    }
-
-    /**
-     * Does the RecordReader need to include this set of records?
-     * @return true unless none of the rows qualify
-     */
-    public boolean isNeeded() {
-      switch (this) {
-        case NO:
-        case NULL:
-        case NO_NULL:
-          return false;
-        default:
-          return true;
-      }
-    }
-  }
-
-  /**
-   * Get the leaf predicates that are required to evaluate the predicate. The
-   * list will have the duplicates removed.
-   * @return the list of leaf predicates
-   */
-  public List<PredicateLeaf> getLeaves();
-
-  /**
-   * Get the expression tree. This should only needed for file formats that
-   * need to translate the expression to an internal form.
-   */
-  public ExpressionTree getExpression();
- 
-  /**
-   * Evaluate the entire predicate based on the values for the leaf predicates.
-   * @param leaves the value of each leaf predicate
-   * @return the value of hte entire predicate
-   */
-  public TruthValue evaluate(TruthValue[] leaves);
-
-  /**
-   * Serialize the SARG as a kyro object and return the base64 string.
-   *
-   * Hive should replace the current XML-based AST serialization for predicate pushdown
-   * with the Kryo serialization of the SARG because the representation is much more
-   * compact and focused on what is needed for predicate pushdown.
-   *
-   * @return the serialized SARG
-   */
-  public String toKryo();
-
-  /**
-   * A builder object for contexts outside of Hive where it isn't easy to
-   * get a ExprNodeDesc. The user must call startOr, startAnd, or startNot
-   * before adding any leaves.
-   */
-  public interface Builder {
-
-    /**
-     * Start building an or operation and push it on the stack.
-     * @return this
-     */
-    public Builder startOr();
-
-    /**
-     * Start building an and operation and push it on the stack.
-     * @return this
-     */
-    public Builder startAnd();
-
-    /**
-     * Start building a not operation and push it on the stack.
-     * @return this
-     */
-    public Builder startNot();
-
-    /**
-     * Finish the current operation and pop it off of the stack. Each start
-     * call must have a matching end.
-     * @return this
-     */
-    public Builder end();
-
-    /**
-     * Add a less than leaf to the current item on the stack.
-     * @param column the name of the column
-     * @param type the type of the expression
-     * @param literal the literal
-     * @return this
-     */
-    public Builder lessThan(String column, PredicateLeaf.Type type,
-                            Object literal);
-
-    /**
-     * Add a less than equals leaf to the current item on the stack.
-     * @param column the name of the column
-     * @param type the type of the expression
-     * @param literal the literal
-     * @return this
-     */
-    public Builder lessThanEquals(String column, PredicateLeaf.Type type,
-                                  Object literal);
-
-    /**
-     * Add an equals leaf to the current item on the stack.
-     * @param column the name of the column
-     * @param type the type of the expression
-     * @param literal the literal
-     * @return this
-     */
-    public Builder equals(String column, PredicateLeaf.Type type,
-                          Object literal);
-
-    /**
-     * Add a null safe equals leaf to the current item on the stack.
-     * @param column the name of the column
-     * @param type the type of the expression
-     * @param literal the literal
-     * @return this
-     */
-    public Builder nullSafeEquals(String column, PredicateLeaf.Type type,
-                                  Object literal);
-
-    /**
-     * Add an in leaf to the current item on the stack.
-     * @param column the name of the column
-     * @param type the type of the expression
-     * @param literal the literal
-     * @return this
-     */
-    public Builder in(String column, PredicateLeaf.Type type,
-                      Object... literal);
-
-    /**
-     * Add an is null leaf to the current item on the stack.
-     * @param column the name of the column
-     * @param type the type of the expression
-     * @return this
-     */
-    public Builder isNull(String column, PredicateLeaf.Type type);
-
-    /**
-     * Add a between leaf to the current item on the stack.
-     * @param column the name of the column
-     * @param type the type of the expression
-     * @param lower the literal
-     * @param upper the literal
-     * @return this
-     */
-    public Builder between(String column, PredicateLeaf.Type type,
-                           Object lower, Object upper);
-
-    /**
-     * Add a truth value to the expression.
-     * @param truth
-     * @return this
-     */
-    public Builder literal(TruthValue truth);
-
-    /**
-     * Build and return the SearchArgument that has been defined. All of the
-     * starts must have been ended before this call.
-     * @return the new SearchArgument
-     */
-    public SearchArgument build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/9ae70cb4/serde/src/java/org/apache/hadoop/hive/serde2/io/HiveDecimalWritable.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/io/HiveDecimalWritable.java b/serde/src/java/org/apache/hadoop/hive/serde2/io/HiveDecimalWritable.java
deleted file mode 100644
index 0578d24..0000000
--- a/serde/src/java/org/apache/hadoop/hive/serde2/io/HiveDecimalWritable.java
+++ /dev/null
@@ -1,174 +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.
- */
-package org.apache.hadoop.hive.serde2.io;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.math.BigInteger;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hive.common.type.HiveDecimal;
-
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.io.WritableUtils;
-
-public class HiveDecimalWritable implements WritableComparable<HiveDecimalWritable> {
-
-  static final private Log LOG = LogFactory.getLog(HiveDecimalWritable.class);
-
-  private byte[] internalStorage = new byte[0];
-  private int scale;
-
-  public HiveDecimalWritable() {
-  }
-
-  public HiveDecimalWritable(String value) {
-    set(HiveDecimal.create(value));
-  }
-
-  public HiveDecimalWritable(byte[] bytes, int scale) {
-    set(bytes, scale);
-  }
-
-  public HiveDecimalWritable(HiveDecimalWritable writable) {
-    set(writable.getHiveDecimal());
-  }
-
-  public HiveDecimalWritable(HiveDecimal value) {
-    set(value);
-  }
-
-  public HiveDecimalWritable(long value) {
-    set((HiveDecimal.create(value)));
-  }
-
-  public void set(HiveDecimal value) {
-    set(value.unscaledValue().toByteArray(), value.scale());
-  }
-
-  public void set(HiveDecimal value, int maxPrecision, int maxScale) {
-    set(HiveDecimal.enforcePrecisionScale(value, maxPrecision, maxScale));
-  }
-
-  public void set(HiveDecimalWritable writable) {
-    set(writable.getHiveDecimal());
-  }
-
-  public void set(byte[] bytes, int scale) {
-    this.internalStorage = bytes;
-    this.scale = scale;
-  }
-
-  public HiveDecimal getHiveDecimal() {
-    return HiveDecimal.create(new BigInteger(internalStorage), scale);
-  }
-
-  /**
-   * Get a HiveDecimal instance from the writable and constraint it with maximum precision/scale.
-   *
-   * @param maxPrecision maximum precision
-   * @param maxScale maximum scale
-   * @return HiveDecimal instance
-   */
-  public HiveDecimal getHiveDecimal(int maxPrecision, int maxScale) {
-     return HiveDecimal.enforcePrecisionScale(HiveDecimal.
-             create(new BigInteger(internalStorage), scale),
-         maxPrecision, maxScale);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    scale = WritableUtils.readVInt(in);
-    int byteArrayLen = WritableUtils.readVInt(in);
-    if (internalStorage.length != byteArrayLen) {
-      internalStorage = new byte[byteArrayLen];
-    }
-    in.readFully(internalStorage);
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    WritableUtils.writeVInt(out, scale);
-    WritableUtils.writeVInt(out, internalStorage.length);
-    out.write(internalStorage);
-  }
-
-  @Override
-  public int compareTo(HiveDecimalWritable that) {
-    return getHiveDecimal().compareTo(that.getHiveDecimal());
-  }
-
-  @Override
-  public String toString() {
-    return getHiveDecimal().toString();
-  }
-
-  @Override
-  public boolean equals(Object other) {
-    if (this == other) {
-      return true;
-    }
-    if (other == null || getClass() != other.getClass()) {
-      return false;
-    }
-    HiveDecimalWritable bdw = (HiveDecimalWritable) other;
-
-    // 'equals' and 'compareTo' are not compatible with HiveDecimals. We want
-    // compareTo which returns true iff the numbers are equal (e.g.: 3.14 is
-    // the same as 3.140). 'Equals' returns true iff equal and the same scale
-    // is set in the decimals (e.g.: 3.14 is not the same as 3.140)
-    return getHiveDecimal().compareTo(bdw.getHiveDecimal()) == 0;
-  }
-
-  @Override
-  public int hashCode() {
-    return getHiveDecimal().hashCode();
-  }
-
-  /* (non-Javadoc)
-   * In order to update a Decimal128 fast (w/o allocation) we need to expose access to the
-   * internal storage bytes and scale.  
-   * @return
-   */
-  public byte[] getInternalStorage() {
-    return internalStorage;
-  }
-  
-  /* (non-Javadoc)
-   * In order to update a Decimal128 fast (w/o allocation) we need to expose access to the
-   * internal storage bytes and scale.  
-   */
-  public int getScale() {
-    return scale;
-  }
-
-  public static
-  HiveDecimalWritable enforcePrecisionScale(HiveDecimalWritable writable,
-                                            int precision, int scale) {
-    if (writable == null) {
-      return null;
-    }
-
-    HiveDecimal dec =
-        HiveDecimal.enforcePrecisionScale(writable.getHiveDecimal(), precision,
-            scale);
-    return dec == null ? null : new HiveDecimalWritable(dec);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/9ae70cb4/storage-api/pom.xml
----------------------------------------------------------------------
diff --git a/storage-api/pom.xml b/storage-api/pom.xml
new file mode 100644
index 0000000..71b51b8
--- /dev/null
+++ b/storage-api/pom.xml
@@ -0,0 +1,85 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.hive</groupId>
+    <artifactId>hive</artifactId>
+    <version>2.0.0-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <artifactId>hive-storage-api</artifactId>
+  <packaging>jar</packaging>
+  <name>Hive Storage API</name>
+
+  <properties>
+    <hive.path.to.root>..</hive.path.to.root>
+  </properties>
+
+  <dependencies>
+    <!-- dependencies are always listed in sorted order by groupId, artifectId -->
+    <!-- inter-project -->
+    <dependency>
+      <groupId>log4j</groupId>
+      <artifactId>log4j</artifactId>
+      <version>${log4j.version}</version>
+    </dependency>
+    <!-- test inter-project -->
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <version>${junit.version}</version>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+
+  <profiles>
+    <profile>
+      <id>hadoop-1</id>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-core</artifactId>
+          <version>${hadoop-20S.version}</version>
+         <optional>true</optional>
+        </dependency>
+      </dependencies>
+    </profile>
+   <profile>
+      <id>hadoop-2</id>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-common</artifactId>
+          <version>${hadoop-23.version}</version>
+          <optional>true</optional>
+        </dependency>
+      </dependencies>
+    </profile>
+  </profiles>
+
+  <build>
+    <sourceDirectory>${basedir}/src/java</sourceDirectory>
+    <testSourceDirectory>${basedir}/src/test</testSourceDirectory>
+    <testResources>
+      <testResource>
+        <directory>${basedir}/src/test/resources</directory>
+      </testResource>
+    </testResources>
+  </build>
+</project>

http://git-wip-us.apache.org/repos/asf/hive/blob/9ae70cb4/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveDecimal.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveDecimal.java b/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveDecimal.java
new file mode 100644
index 0000000..7d7fb28
--- /dev/null
+++ b/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveDecimal.java
@@ -0,0 +1,312 @@
+/**
+ * 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.common.type;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.math.RoundingMode;
+
+/**
+ *
+ * HiveDecimal. Simple wrapper for BigDecimal. Adds fixed max precision and non scientific string
+ * representation
+ *
+ */
+public class HiveDecimal implements Comparable<HiveDecimal> {
+  public static final int MAX_PRECISION = 38;
+  public static final int MAX_SCALE = 38;
+
+  /**
+   * Default precision/scale when user doesn't specify in the column metadata, such as
+   * decimal and decimal(8).
+   */
+  public static final int USER_DEFAULT_PRECISION = 10;
+  public static final int USER_DEFAULT_SCALE = 0;
+
+  /**
+   *  Default precision/scale when system is not able to determine them, such as in case
+   *  of a non-generic udf.
+   */
+  public static final int SYSTEM_DEFAULT_PRECISION = 38;
+  public static final int SYSTEM_DEFAULT_SCALE = 18;
+
+  public static final HiveDecimal ZERO = new HiveDecimal(BigDecimal.ZERO);
+  public static final HiveDecimal ONE = new HiveDecimal(BigDecimal.ONE);
+
+  public static final int ROUND_FLOOR = BigDecimal.ROUND_FLOOR;
+  public static final int ROUND_CEILING = BigDecimal.ROUND_CEILING;
+  public static final int ROUND_HALF_UP = BigDecimal.ROUND_HALF_UP;
+
+  private BigDecimal bd = BigDecimal.ZERO;
+
+  private HiveDecimal(BigDecimal bd) {
+    this.bd = bd;
+  }
+
+  public static HiveDecimal create(BigDecimal b) {
+    return create(b, true);
+  }
+
+  public static HiveDecimal create(BigDecimal b, boolean allowRounding) {
+    BigDecimal bd = normalize(b, allowRounding);
+    return bd == null ? null : new HiveDecimal(bd);
+  }
+
+  public static HiveDecimal create(BigInteger unscaled, int scale) {
+    BigDecimal bd = normalize(new BigDecimal(unscaled, scale), true);
+    return bd == null ? null : new HiveDecimal(bd);
+  }
+
+  public static HiveDecimal create(String dec) {
+    BigDecimal bd;
+    try {
+      bd = new BigDecimal(dec.trim());
+    } catch (NumberFormatException ex) {
+      return null;
+    }
+
+    bd = normalize(bd, true);
+    return bd == null ? null : new HiveDecimal(bd);
+  }
+
+  public static HiveDecimal create(BigInteger bi) {
+    BigDecimal bd = normalize(new BigDecimal(bi), true);
+    return bd == null ? null : new HiveDecimal(bd);
+  }
+
+  public static HiveDecimal create(int i) {
+    return new HiveDecimal(new BigDecimal(i));
+  }
+
+  public static HiveDecimal create(long l) {
+    return new HiveDecimal(new BigDecimal(l));
+  }
+
+  @Override
+  public String toString() {
+     return bd.toPlainString();
+  }
+
+  public HiveDecimal setScale(int i) {
+    return new HiveDecimal(bd.setScale(i, RoundingMode.HALF_UP));
+  }
+
+  @Override
+  public int compareTo(HiveDecimal dec) {
+    return bd.compareTo(dec.bd);
+  }
+
+  @Override
+  public int hashCode() {
+    return bd.hashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj == null || obj.getClass() != getClass()) {
+      return false;
+    }
+    return bd.equals(((HiveDecimal) obj).bd);
+  }
+
+  public int scale() {
+    return bd.scale();
+  }
+
+  /**
+   * Returns the number of digits (integer and fractional) in the number, which is equivalent
+   * to SQL decimal precision. Note that this is different from BigDecimal.precision(),
+   * which returns the precision of the unscaled value (BigDecimal.valueOf(0.01).precision() = 1,
+   * whereas HiveDecimal.create("0.01").precision() = 2).
+   * If you want the BigDecimal precision, use HiveDecimal.bigDecimalValue().precision()
+   * @return
+   */
+  public int precision() {
+    int bdPrecision = bd.precision();
+    int bdScale = bd.scale();
+
+    if (bdPrecision < bdScale) {
+      // This can happen for numbers less than 0.1
+      // For 0.001234: bdPrecision=4, bdScale=6
+      // In this case, we'll set the type to have the same precision as the scale.
+      return bdScale;
+    }
+    return bdPrecision;
+  }
+
+  public int intValue() {
+    return bd.intValue();
+  }
+
+  public double doubleValue() {
+    return bd.doubleValue();
+  }
+
+  public long longValue() {
+    return bd.longValue();
+  }
+
+  public short shortValue() {
+    return bd.shortValue();
+  }
+
+  public float floatValue() {
+    return bd.floatValue();
+  }
+
+  public BigDecimal bigDecimalValue() {
+    return bd;
+  }
+
+  public byte byteValue() {
+    return bd.byteValue();
+  }
+
+  public HiveDecimal setScale(int adjustedScale, int rm) {
+    return create(bd.setScale(adjustedScale, rm));
+  }
+
+  public HiveDecimal subtract(HiveDecimal dec) {
+    return create(bd.subtract(dec.bd));
+  }
+
+  public HiveDecimal multiply(HiveDecimal dec) {
+    return create(bd.multiply(dec.bd), false);
+  }
+
+  public BigInteger unscaledValue() {
+    return bd.unscaledValue();
+  }
+
+  public HiveDecimal scaleByPowerOfTen(int n) {
+    return create(bd.scaleByPowerOfTen(n));
+  }
+
+  public HiveDecimal abs() {
+    return create(bd.abs());
+  }
+
+  public HiveDecimal negate() {
+    return create(bd.negate());
+  }
+
+  public HiveDecimal add(HiveDecimal dec) {
+    return create(bd.add(dec.bd));
+  }
+
+  public HiveDecimal pow(int n) {
+    BigDecimal result = normalize(bd.pow(n), false);
+    return result == null ? null : new HiveDecimal(result);
+  }
+
+  public HiveDecimal remainder(HiveDecimal dec) {
+    return create(bd.remainder(dec.bd));
+  }
+
+  public HiveDecimal divide(HiveDecimal dec) {
+    return create(bd.divide(dec.bd, MAX_SCALE, RoundingMode.HALF_UP), true);
+  }
+
+  /**
+   * Get the sign of the underlying decimal.
+   * @return 0 if the decimal is equal to 0, -1 if less than zero, and 1 if greater than 0
+   */
+  public int signum() {
+    return bd.signum();
+  }
+
+  private static BigDecimal trim(BigDecimal d) {
+    if (d.compareTo(BigDecimal.ZERO) == 0) {
+      // Special case for 0, because java doesn't strip zeros correctly on that number.
+      d = BigDecimal.ZERO;
+    } else {
+      d = d.stripTrailingZeros();
+      if (d.scale() < 0) {
+        // no negative scale decimals
+        d = d.setScale(0);
+      }
+    }
+    return d;
+  }
+
+  private static BigDecimal normalize(BigDecimal bd, boolean allowRounding) {
+    if (bd == null) {
+      return null;
+    }
+
+    bd = trim(bd);
+
+    int intDigits = bd.precision() - bd.scale();
+
+    if (intDigits > MAX_PRECISION) {
+      return null;
+    }
+
+    int maxScale = Math.min(MAX_SCALE, Math.min(MAX_PRECISION - intDigits, bd.scale()));
+    if (bd.scale() > maxScale ) {
+      if (allowRounding) {
+        bd = bd.setScale(maxScale, RoundingMode.HALF_UP);
+        // Trimming is again necessary, because rounding may introduce new trailing 0's.
+        bd = trim(bd);
+      } else {
+        bd = null;
+      }
+    }
+
+    return bd;
+  }
+
+  public static BigDecimal enforcePrecisionScale(BigDecimal bd, int maxPrecision, int maxScale) {
+    if (bd == null) {
+      return null;
+    }
+
+    bd = trim(bd);
+
+    if (bd.scale() > maxScale) {
+      bd = bd.setScale(maxScale, RoundingMode.HALF_UP);
+    }
+
+    int maxIntDigits = maxPrecision - maxScale;
+    int intDigits = bd.precision() - bd.scale();
+    if (intDigits > maxIntDigits) {
+      return null;
+    }
+
+    return bd;
+  }
+
+  public static HiveDecimal enforcePrecisionScale(HiveDecimal dec, int maxPrecision, int maxScale) {
+    if (dec == null) {
+      return null;
+    }
+
+    // Minor optimization, avoiding creating new objects.
+    if (dec.precision() - dec.scale() <= maxPrecision - maxScale &&
+        dec.scale() <= maxScale) {
+      return dec;
+    }
+
+    BigDecimal bd = enforcePrecisionScale(dec.bd, maxPrecision, maxScale);
+    if (bd == null) {
+      return null;
+    }
+
+    return HiveDecimal.create(bd);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/9ae70cb4/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/BytesColumnVector.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/BytesColumnVector.java b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/BytesColumnVector.java
new file mode 100644
index 0000000..02c52fa
--- /dev/null
+++ b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/BytesColumnVector.java
@@ -0,0 +1,322 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector;
+
+/**
+ * This class supports string and binary data by value reference -- i.e. each field is
+ * explicitly present, as opposed to provided by a dictionary reference.
+ * In some cases, all the values will be in the same byte array to begin with,
+ * but this need not be the case. If each value is in a separate byte
+ * array to start with, or not all of the values are in the same original
+ * byte array, you can still assign data by reference into this column vector.
+ * This gives flexibility to use this in multiple situations.
+ * <p>
+ * When setting data by reference, the caller
+ * is responsible for allocating the byte arrays used to hold the data.
+ * You can also set data by value, as long as you call the initBuffer() method first.
+ * You can mix "by value" and "by reference" in the same column vector,
+ * though that use is probably not typical.
+ */
+public class BytesColumnVector extends ColumnVector {
+  public byte[][] vector;
+  public int[] start;          // start offset of each field
+
+  /*
+   * The length of each field. If the value repeats for every entry, then it is stored
+   * in vector[0] and isRepeating from the superclass is set to true.
+   */
+  public int[] length;
+  private byte[] buffer;   // optional buffer to use when actually copying in data
+  private int nextFree;    // next free position in buffer
+
+  // Estimate that there will be 16 bytes per entry
+  static final int DEFAULT_BUFFER_SIZE = 16 * VectorizedRowBatch.DEFAULT_SIZE;
+
+  // Proportion of extra space to provide when allocating more buffer space.
+  static final float EXTRA_SPACE_FACTOR = (float) 1.2;
+
+  /**
+   * Use this constructor for normal operation.
+   * All column vectors should be the default size normally.
+   */
+  public BytesColumnVector() {
+    this(VectorizedRowBatch.DEFAULT_SIZE);
+  }
+
+  /**
+   * Don't call this constructor except for testing purposes.
+   *
+   * @param size  number of elements in the column vector
+   */
+  public BytesColumnVector(int size) {
+    super(size);
+    vector = new byte[size][];
+    start = new int[size];
+    length = new int[size];
+  }
+
+  /**
+   * Additional reset work for BytesColumnVector (releasing scratch bytes for by value strings).
+   */
+  @Override
+  public void reset() {
+    super.reset();
+    initBuffer(0);
+  }
+
+  /** Set a field by reference.
+   *
+   * @param elementNum index within column vector to set
+   * @param sourceBuf container of source data
+   * @param start start byte position within source
+   * @param length  length of source byte sequence
+   */
+  public void setRef(int elementNum, byte[] sourceBuf, int start, int length) {
+    vector[elementNum] = sourceBuf;
+    this.start[elementNum] = start;
+    this.length[elementNum] = length;
+  }
+
+  /**
+   * You must call initBuffer first before using setVal().
+   * Provide the estimated number of bytes needed to hold
+   * a full column vector worth of byte string data.
+   *
+   * @param estimatedValueSize  Estimated size of buffer space needed
+   */
+  public void initBuffer(int estimatedValueSize) {
+    nextFree = 0;
+
+    // if buffer is already allocated, keep using it, don't re-allocate
+    if (buffer != null) {
+      return;
+    }
+
+    // allocate a little extra space to limit need to re-allocate
+    int bufferSize = this.vector.length * (int)(estimatedValueSize * EXTRA_SPACE_FACTOR);
+    if (bufferSize < DEFAULT_BUFFER_SIZE) {
+      bufferSize = DEFAULT_BUFFER_SIZE;
+    }
+    buffer = new byte[bufferSize];
+  }
+
+  /**
+   * Initialize buffer to default size.
+   */
+  public void initBuffer() {
+    initBuffer(0);
+  }
+
+  /**
+   * @return amount of buffer space currently allocated
+   */
+  public int bufferSize() {
+    if (buffer == null) {
+      return 0;
+    }
+    return buffer.length;
+  }
+
+  /**
+   * Set a field by actually copying in to a local buffer.
+   * If you must actually copy data in to the array, use this method.
+   * DO NOT USE this method unless it's not practical to set data by reference with setRef().
+   * Setting data by reference tends to run a lot faster than copying data in.
+   *
+   * @param elementNum index within column vector to set
+   * @param sourceBuf container of source data
+   * @param start start byte position within source
+   * @param length  length of source byte sequence
+   */
+  public void setVal(int elementNum, byte[] sourceBuf, int start, int length) {
+    if ((nextFree + length) > buffer.length) {
+      increaseBufferSpace(length);
+    }
+    System.arraycopy(sourceBuf, start, buffer, nextFree, length);
+    vector[elementNum] = buffer;
+    this.start[elementNum] = nextFree;
+    this.length[elementNum] = length;
+    nextFree += length;
+  }
+
+  /**
+   * Set a field to the concatenation of two string values. Result data is copied
+   * into the internal buffer.
+   *
+   * @param elementNum index within column vector to set
+   * @param leftSourceBuf container of left argument
+   * @param leftStart start of left argument
+   * @param leftLen length of left argument
+   * @param rightSourceBuf container of right argument
+   * @param rightStart start of right argument
+   * @param rightLen length of right arugment
+   */
+  public void setConcat(int elementNum, byte[] leftSourceBuf, int leftStart, int leftLen,
+      byte[] rightSourceBuf, int rightStart, int rightLen) {
+    int newLen = leftLen + rightLen;
+    if ((nextFree + newLen) > buffer.length) {
+      increaseBufferSpace(newLen);
+    }
+    vector[elementNum] = buffer;
+    this.start[elementNum] = nextFree;
+    this.length[elementNum] = newLen;
+
+    System.arraycopy(leftSourceBuf, leftStart, buffer, nextFree, leftLen);
+    nextFree += leftLen;
+    System.arraycopy(rightSourceBuf, rightStart, buffer, nextFree, rightLen);
+    nextFree += rightLen;
+  }
+
+  /**
+   * Increase buffer space enough to accommodate next element.
+   * This uses an exponential increase mechanism to rapidly
+   * increase buffer size to enough to hold all data.
+   * As batches get re-loaded, buffer space allocated will quickly
+   * stabilize.
+   *
+   * @param nextElemLength size of next element to be added
+   */
+  public void increaseBufferSpace(int nextElemLength) {
+
+    // Keep doubling buffer size until there will be enough space for next element.
+    int newLength = 2 * buffer.length;
+    while((nextFree + nextElemLength) > newLength) {
+      newLength *= 2;
+    }
+
+    // Allocate new buffer, copy data to it, and set buffer to new buffer.
+    byte[] newBuffer = new byte[newLength];
+    System.arraycopy(buffer, 0, newBuffer, 0, nextFree);
+    buffer = newBuffer;
+  }
+
+  /** Copy the current object contents into the output. Only copy selected entries,
+    * as indicated by selectedInUse and the sel array.
+    */
+  public void copySelected(
+      boolean selectedInUse, int[] sel, int size, BytesColumnVector output) {
+
+    // Output has nulls if and only if input has nulls.
+    output.noNulls = noNulls;
+    output.isRepeating = false;
+
+    // Handle repeating case
+    if (isRepeating) {
+      output.setVal(0, vector[0], start[0], length[0]);
+      output.isNull[0] = isNull[0];
+      output.isRepeating = true;
+      return;
+    }
+
+    // Handle normal case
+
+    // Copy data values over
+    if (selectedInUse) {
+      for (int j = 0; j < size; j++) {
+        int i = sel[j];
+        output.setVal(i, vector[i], start[i], length[i]);
+      }
+    }
+    else {
+      for (int i = 0; i < size; i++) {
+        output.setVal(i, vector[i], start[i], length[i]);
+      }
+    }
+
+    // Copy nulls over if needed
+    if (!noNulls) {
+      if (selectedInUse) {
+        for (int j = 0; j < size; j++) {
+          int i = sel[j];
+          output.isNull[i] = isNull[i];
+        }
+      }
+      else {
+        System.arraycopy(isNull, 0, output.isNull, 0, size);
+      }
+    }
+  }
+
+  /** Simplify vector by brute-force flattening noNulls and isRepeating
+    * This can be used to reduce combinatorial explosion of code paths in VectorExpressions
+    * with many arguments, at the expense of loss of some performance.
+    */
+  public void flatten(boolean selectedInUse, int[] sel, int size) {
+    flattenPush();
+    if (isRepeating) {
+      isRepeating = false;
+
+      // setRef is used below and this is safe, because the reference
+      // is to data owned by this column vector. If this column vector
+      // gets re-used, the whole thing is re-used together so there
+      // is no danger of a dangling reference.
+
+      // Only copy data values if entry is not null. The string value
+      // at position 0 is undefined if the position 0 value is null.
+      if (noNulls || !isNull[0]) {
+
+        // loops start at position 1 because position 0 is already set
+        if (selectedInUse) {
+          for (int j = 1; j < size; j++) {
+            int i = sel[j];
+            this.setRef(i, vector[0], start[0], length[0]);
+          }
+        } else {
+          for (int i = 1; i < size; i++) {
+            this.setRef(i, vector[0], start[0], length[0]);
+          }
+        }
+      }
+      flattenRepeatingNulls(selectedInUse, sel, size);
+    }
+    flattenNoNulls(selectedInUse, sel, size);
+  }
+
+  // Fill the all the vector entries with provided value
+  public void fill(byte[] value) {
+    noNulls = true;
+    isRepeating = true;
+    setRef(0, value, 0, value.length);
+  }
+
+  @Override
+  public void setElement(int outElementNum, int inputElementNum, ColumnVector inputVector) {
+    BytesColumnVector in = (BytesColumnVector) inputVector;
+    setVal(outElementNum, in.vector[inputElementNum], in.start[inputElementNum], in.length[inputElementNum]);
+  }
+
+  @Override
+  public void init() {
+    initBuffer(0);
+  }
+
+  @Override
+  public void stringifyValue(StringBuilder buffer, int row) {
+    if (isRepeating) {
+      row = 0;
+    }
+    if (noNulls || !isNull[row]) {
+      buffer.append('"');
+      buffer.append(new String(this.buffer, start[row], length[row]));
+      buffer.append('"');
+    } else {
+      buffer.append("null");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/9ae70cb4/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/ColumnVector.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/ColumnVector.java b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/ColumnVector.java
new file mode 100644
index 0000000..cb75c2c
--- /dev/null
+++ b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/ColumnVector.java
@@ -0,0 +1,173 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector;
+
+import java.util.Arrays;
+
+/**
+ * ColumnVector contains the shared structure for the sub-types,
+ * including NULL information, and whether this vector
+ * repeats, i.e. has all values the same, so only the first
+ * one is set. This is used to accelerate query performance
+ * by handling a whole vector in O(1) time when applicable.
+ *
+ * The fields are public by design since this is a performance-critical
+ * structure that is used in the inner loop of query execution.
+ */
+public abstract class ColumnVector {
+
+  /*
+   * The current kinds of column vectors.
+   */
+  public static enum Type {
+    LONG,
+    DOUBLE,
+    BYTES,
+    DECIMAL
+  }
+
+  /*
+   * If hasNulls is true, then this array contains true if the value
+   * is null, otherwise false. The array is always allocated, so a batch can be re-used
+   * later and nulls added.
+   */
+  public boolean[] isNull;
+
+  // If the whole column vector has no nulls, this is true, otherwise false.
+  public boolean noNulls;
+
+  /*
+   * True if same value repeats for whole column vector.
+   * If so, vector[0] holds the repeating value.
+   */
+  public boolean isRepeating;
+
+  // Variables to hold state from before flattening so it can be easily restored.
+  private boolean preFlattenIsRepeating;
+  private boolean preFlattenNoNulls;
+
+  /**
+   * Constructor for super-class ColumnVector. This is not called directly,
+   * but used to initialize inherited fields.
+   *
+   * @param len Vector length
+   */
+  public ColumnVector(int len) {
+    isNull = new boolean[len];
+    noNulls = true;
+    isRepeating = false;
+  }
+
+  /**
+     * Resets the column to default state
+     *  - fills the isNull array with false
+     *  - sets noNulls to true
+     *  - sets isRepeating to false
+     */
+    public void reset() {
+      if (false == noNulls) {
+        Arrays.fill(isNull, false);
+      }
+      noNulls = true;
+      isRepeating = false;
+    }
+
+    abstract public void flatten(boolean selectedInUse, int[] sel, int size);
+
+    // Simplify vector by brute-force flattening noNulls if isRepeating
+    // This can be used to reduce combinatorial explosion of code paths in VectorExpressions
+    // with many arguments.
+    public void flattenRepeatingNulls(boolean selectedInUse, int[] sel, int size) {
+
+      boolean nullFillValue;
+
+      if (noNulls) {
+        nullFillValue = false;
+      } else {
+        nullFillValue = isNull[0];
+      }
+
+      if (selectedInUse) {
+        for (int j = 0; j < size; j++) {
+          int i = sel[j];
+          isNull[i] = nullFillValue;
+        }
+      } else {
+        Arrays.fill(isNull, 0, size, nullFillValue);
+      }
+
+      // all nulls are now explicit
+      noNulls = false;
+    }
+
+    public void flattenNoNulls(boolean selectedInUse, int[] sel, int size) {
+      if (noNulls) {
+        noNulls = false;
+        if (selectedInUse) {
+          for (int j = 0; j < size; j++) {
+            int i = sel[j];
+            isNull[i] = false;
+          }
+        } else {
+          Arrays.fill(isNull, 0, size, false);
+        }
+      }
+    }
+
+    /**
+     * Restore the state of isRepeating and noNulls to what it was
+     * before flattening. This must only be called just after flattening
+     * and then evaluating a VectorExpression on the column vector.
+     * It is an optimization that allows other operations on the same
+     * column to continue to benefit from the isRepeating and noNulls
+     * indicators.
+     */
+    public void unFlatten() {
+      isRepeating = preFlattenIsRepeating;
+      noNulls = preFlattenNoNulls;
+    }
+
+    // Record repeating and no nulls state to be restored later.
+    protected void flattenPush() {
+      preFlattenIsRepeating = isRepeating;
+      preFlattenNoNulls = noNulls;
+    }
+
+    /**
+     * Set the element in this column vector from the given input vector.
+     */
+    public abstract void setElement(int outElementNum, int inputElementNum, ColumnVector inputVector);
+
+    /**
+     * Initialize the column vector. This method can be overridden by specific column vector types.
+     * Use this method only if the individual type of the column vector is not known, otherwise its
+     * preferable to call specific initialization methods.
+     */
+    public void init() {
+      // Do nothing by default
+    }
+
+    /**
+     * Print the value for this column into the given string builder.
+     * @param buffer the buffer to print into
+     * @param row the id of the row to print
+     */
+    public abstract void stringifyValue(StringBuilder buffer,
+                                        int row);
+  }

http://git-wip-us.apache.org/repos/asf/hive/blob/9ae70cb4/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/DecimalColumnVector.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/DecimalColumnVector.java b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/DecimalColumnVector.java
new file mode 100644
index 0000000..74a9d5f
--- /dev/null
+++ b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/DecimalColumnVector.java
@@ -0,0 +1,106 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector;
+
+import java.math.BigInteger;
+
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
+
+public class DecimalColumnVector extends ColumnVector {
+
+  /**
+   * A vector of HiveDecimalWritable objects.
+   *
+   * For high performance and easy access to this low-level structure,
+   * the fields are public by design (as they are in other ColumnVector
+   * types).
+   */
+  public HiveDecimalWritable[] vector;
+  public short scale;
+  public short precision;
+
+  public DecimalColumnVector(int precision, int scale) {
+    this(VectorizedRowBatch.DEFAULT_SIZE, precision, scale);
+  }
+
+  public DecimalColumnVector(int size, int precision, int scale) {
+    super(size);
+    this.precision = (short) precision;
+    this.scale = (short) scale;
+    vector = new HiveDecimalWritable[size];
+    for (int i = 0; i < size; i++) {
+      vector[i] = new HiveDecimalWritable(HiveDecimal.ZERO);
+    }
+  }
+
+  @Override
+  public void flatten(boolean selectedInUse, int[] sel, int size) {
+    // TODO Auto-generated method stub
+  }
+
+  @Override
+  public void setElement(int outElementNum, int inputElementNum, ColumnVector inputVector) {
+    HiveDecimal hiveDec = ((DecimalColumnVector) inputVector).vector[inputElementNum].getHiveDecimal(precision, scale);
+    if (hiveDec == null) {
+      noNulls = false;
+      isNull[outElementNum] = true;
+    } else {
+      vector[outElementNum].set(hiveDec);
+    }
+  }
+
+  @Override
+  public void stringifyValue(StringBuilder buffer, int row) {
+    if (isRepeating) {
+      row = 0;
+    }
+    if (noNulls || !isNull[row]) {
+      buffer.append(vector[row].toString());
+    } else {
+      buffer.append("null");
+    }
+  }
+
+  public void set(int elementNum, HiveDecimalWritable writeable) {
+    HiveDecimal hiveDec = writeable.getHiveDecimal(precision, scale);
+    if (hiveDec == null) {
+      noNulls = false;
+      isNull[elementNum] = true;
+    } else {
+      vector[elementNum].set(hiveDec);
+    }
+  }
+
+  public void set(int elementNum, HiveDecimal hiveDec) {
+    HiveDecimal checkedDec = HiveDecimal.enforcePrecisionScale(hiveDec, precision, scale);
+    if (checkedDec == null) {
+      noNulls = false;
+      isNull[elementNum] = true;
+    } else {
+      vector[elementNum].set(checkedDec);
+    }
+  }
+
+  public void setNullDataValue(int elementNum) {
+    // E.g. For scale 2 the minimum is "0.01"
+    HiveDecimal minimumNonZeroValue = HiveDecimal.create(BigInteger.ONE, scale);
+    vector[elementNum].set(minimumNonZeroValue);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/9ae70cb4/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/DoubleColumnVector.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/DoubleColumnVector.java b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/DoubleColumnVector.java
new file mode 100644
index 0000000..4a7811d
--- /dev/null
+++ b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/DoubleColumnVector.java
@@ -0,0 +1,143 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.exec.vector;
+
+import java.util.Arrays;
+
+/**
+ * This class represents a nullable double precision floating point column vector.
+ * This class will be used for operations on all floating point types (float, double)
+ * and as such will use a 64-bit double value to hold the biggest possible value.
+ * During copy-in/copy-out, smaller types (i.e. float) will be converted as needed. This will
+ * reduce the amount of code that needs to be generated and also will run fast since the
+ * machine operates with 64-bit words.
+ *
+ * The vector[] field is public by design for high-performance access in the inner
+ * loop of query execution.
+ */
+public class DoubleColumnVector extends ColumnVector {
+  public double[] vector;
+  public static final double NULL_VALUE = Double.NaN;
+
+  /**
+   * Use this constructor by default. All column vectors
+   * should normally be the default size.
+   */
+  public DoubleColumnVector() {
+    this(VectorizedRowBatch.DEFAULT_SIZE);
+  }
+
+  /**
+   * Don't use this except for testing purposes.
+   *
+   * @param len
+   */
+  public DoubleColumnVector(int len) {
+    super(len);
+    vector = new double[len];
+  }
+
+  // Copy the current object contents into the output. Only copy selected entries,
+  // as indicated by selectedInUse and the sel array.
+  public void copySelected(
+      boolean selectedInUse, int[] sel, int size, DoubleColumnVector output) {
+
+    // Output has nulls if and only if input has nulls.
+    output.noNulls = noNulls;
+    output.isRepeating = false;
+
+    // Handle repeating case
+    if (isRepeating) {
+      output.vector[0] = vector[0];
+      output.isNull[0] = isNull[0];
+      output.isRepeating = true;
+      return;
+    }
+
+    // Handle normal case
+
+    // Copy data values over
+    if (selectedInUse) {
+      for (int j = 0; j < size; j++) {
+        int i = sel[j];
+        output.vector[i] = vector[i];
+      }
+    }
+    else {
+      System.arraycopy(vector, 0, output.vector, 0, size);
+    }
+
+    // Copy nulls over if needed
+    if (!noNulls) {
+      if (selectedInUse) {
+        for (int j = 0; j < size; j++) {
+          int i = sel[j];
+          output.isNull[i] = isNull[i];
+        }
+      }
+      else {
+        System.arraycopy(isNull, 0, output.isNull, 0, size);
+      }
+    }
+  }
+
+  // Fill the column vector with the provided value
+  public void fill(double value) {
+    noNulls = true;
+    isRepeating = true;
+    vector[0] = value;
+  }
+
+  // Simplify vector by brute-force flattening noNulls and isRepeating
+  // This can be used to reduce combinatorial explosion of code paths in VectorExpressions
+  // with many arguments.
+  public void flatten(boolean selectedInUse, int[] sel, int size) {
+    flattenPush();
+    if (isRepeating) {
+      isRepeating = false;
+      double repeatVal = vector[0];
+      if (selectedInUse) {
+        for (int j = 0; j < size; j++) {
+          int i = sel[j];
+          vector[i] = repeatVal;
+        }
+      } else {
+        Arrays.fill(vector, 0, size, repeatVal);
+      }
+      flattenRepeatingNulls(selectedInUse, sel, size);
+    }
+    flattenNoNulls(selectedInUse, sel, size);
+  }
+
+  @Override
+  public void setElement(int outElementNum, int inputElementNum, ColumnVector inputVector) {
+    vector[outElementNum] = ((DoubleColumnVector) inputVector).vector[inputElementNum];
+  }
+
+  @Override
+  public void stringifyValue(StringBuilder buffer, int row) {
+    if (isRepeating) {
+      row = 0;
+    }
+    if (noNulls || !isNull[row]) {
+      buffer.append(vector[row]);
+    } else {
+      buffer.append("null");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/9ae70cb4/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/LongColumnVector.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/LongColumnVector.java b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/LongColumnVector.java
new file mode 100644
index 0000000..5702584
--- /dev/null
+++ b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/LongColumnVector.java
@@ -0,0 +1,189 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.exec.vector;
+
+import java.util.Arrays;
+
+/**
+ * This class represents a nullable int column vector.
+ * This class will be used for operations on all integer types (tinyint, smallint, int, bigint)
+ * and as such will use a 64-bit long value to hold the biggest possible value.
+ * During copy-in/copy-out, smaller int types will be converted as needed. This will
+ * reduce the amount of code that needs to be generated and also will run fast since the
+ * machine operates with 64-bit words.
+ *
+ * The vector[] field is public by design for high-performance access in the inner
+ * loop of query execution.
+ */
+public class LongColumnVector extends ColumnVector {
+  public long[] vector;
+  public static final long NULL_VALUE = 1;
+
+  /**
+   * Use this constructor by default. All column vectors
+   * should normally be the default size.
+   */
+  public LongColumnVector() {
+    this(VectorizedRowBatch.DEFAULT_SIZE);
+  }
+
+  /**
+   * Don't use this except for testing purposes.
+   *
+   * @param len the number of rows
+   */
+  public LongColumnVector(int len) {
+    super(len);
+    vector = new long[len];
+  }
+
+  // Copy the current object contents into the output. Only copy selected entries,
+  // as indicated by selectedInUse and the sel array.
+  public void copySelected(
+      boolean selectedInUse, int[] sel, int size, LongColumnVector output) {
+
+    // Output has nulls if and only if input has nulls.
+    output.noNulls = noNulls;
+    output.isRepeating = false;
+
+    // Handle repeating case
+    if (isRepeating) {
+      output.vector[0] = vector[0];
+      output.isNull[0] = isNull[0];
+      output.isRepeating = true;
+      return;
+    }
+
+    // Handle normal case
+
+    // Copy data values over
+    if (selectedInUse) {
+      for (int j = 0; j < size; j++) {
+        int i = sel[j];
+        output.vector[i] = vector[i];
+      }
+    }
+    else {
+      System.arraycopy(vector, 0, output.vector, 0, size);
+    }
+
+    // Copy nulls over if needed
+    if (!noNulls) {
+      if (selectedInUse) {
+        for (int j = 0; j < size; j++) {
+          int i = sel[j];
+          output.isNull[i] = isNull[i];
+        }
+      }
+      else {
+        System.arraycopy(isNull, 0, output.isNull, 0, size);
+      }
+    }
+  }
+
+  // Copy the current object contents into the output. Only copy selected entries,
+  // as indicated by selectedInUse and the sel array.
+  public void copySelected(
+      boolean selectedInUse, int[] sel, int size, DoubleColumnVector output) {
+
+    // Output has nulls if and only if input has nulls.
+    output.noNulls = noNulls;
+    output.isRepeating = false;
+
+    // Handle repeating case
+    if (isRepeating) {
+      output.vector[0] = vector[0];  // automatic conversion to double is done here
+      output.isNull[0] = isNull[0];
+      output.isRepeating = true;
+      return;
+    }
+
+    // Handle normal case
+
+    // Copy data values over
+    if (selectedInUse) {
+      for (int j = 0; j < size; j++) {
+        int i = sel[j];
+        output.vector[i] = vector[i];
+      }
+    }
+    else {
+      for(int i = 0; i < size; ++i) {
+        output.vector[i] = vector[i];
+      }
+    }
+
+    // Copy nulls over if needed
+    if (!noNulls) {
+      if (selectedInUse) {
+        for (int j = 0; j < size; j++) {
+          int i = sel[j];
+          output.isNull[i] = isNull[i];
+        }
+      }
+      else {
+        System.arraycopy(isNull, 0, output.isNull, 0, size);
+      }
+    }
+  }
+
+  // Fill the column vector with the provided value
+  public void fill(long value) {
+    noNulls = true;
+    isRepeating = true;
+    vector[0] = value;
+  }
+
+  // Simplify vector by brute-force flattening noNulls and isRepeating
+  // This can be used to reduce combinatorial explosion of code paths in VectorExpressions
+  // with many arguments.
+  public void flatten(boolean selectedInUse, int[] sel, int size) {
+    flattenPush();
+    if (isRepeating) {
+      isRepeating = false;
+      long repeatVal = vector[0];
+      if (selectedInUse) {
+        for (int j = 0; j < size; j++) {
+          int i = sel[j];
+          vector[i] = repeatVal;
+        }
+      } else {
+        Arrays.fill(vector, 0, size, repeatVal);
+      }
+      flattenRepeatingNulls(selectedInUse, sel, size);
+    }
+    flattenNoNulls(selectedInUse, sel, size);
+  }
+
+  @Override
+  public void setElement(int outElementNum, int inputElementNum, ColumnVector inputVector) {
+    vector[outElementNum] = ((LongColumnVector) inputVector).vector[inputElementNum];
+  }
+
+  @Override
+  public void stringifyValue(StringBuilder buffer, int row) {
+    if (isRepeating) {
+      row = 0;
+    }
+    if (noNulls || !isNull[row]) {
+      buffer.append(vector[row]);
+    } else {
+      buffer.append("null");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/9ae70cb4/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatch.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatch.java b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatch.java
new file mode 100644
index 0000000..7c18da6
--- /dev/null
+++ b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedRowBatch.java
@@ -0,0 +1,186 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.exec.vector;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * A VectorizedRowBatch is a set of rows, organized with each column
+ * as a vector. It is the unit of query execution, organized to minimize
+ * the cost per row and achieve high cycles-per-instruction.
+ * The major fields are public by design to allow fast and convenient
+ * access by the vectorized query execution code.
+ */
+public class VectorizedRowBatch implements Writable {
+  public int numCols;           // number of columns
+  public ColumnVector[] cols;   // a vector for each column
+  public int size;              // number of rows that qualify (i.e. haven't been filtered out)
+  public int[] selected;        // array of positions of selected values
+  public int[] projectedColumns;
+  public int projectionSize;
+
+  /*
+   * If no filtering has been applied yet, selectedInUse is false,
+   * meaning that all rows qualify. If it is true, then the selected[] array
+   * records the offsets of qualifying rows.
+   */
+  public boolean selectedInUse;
+
+  // If this is true, then there is no data in the batch -- we have hit the end of input.
+  public boolean endOfFile;
+
+  /*
+   * This number is carefully chosen to minimize overhead and typically allows
+   * one VectorizedRowBatch to fit in cache.
+   */
+  public static final int DEFAULT_SIZE = 1024;
+
+  /**
+   * Return a batch with the specified number of columns.
+   * This is the standard constructor -- all batches should be the same size
+   *
+   * @param numCols the number of columns to include in the batch
+   */
+  public VectorizedRowBatch(int numCols) {
+    this(numCols, DEFAULT_SIZE);
+  }
+
+  /**
+   * Return a batch with the specified number of columns and rows.
+   * Only call this constructor directly for testing purposes.
+   * Batch size should normally always be defaultSize.
+   *
+   * @param numCols the number of columns to include in the batch
+   * @param size  the number of rows to include in the batch
+   */
+  public VectorizedRowBatch(int numCols, int size) {
+    this.numCols = numCols;
+    this.size = size;
+    selected = new int[size];
+    selectedInUse = false;
+    this.cols = new ColumnVector[numCols];
+    projectedColumns = new int[numCols];
+
+    // Initially all columns are projected and in the same order
+    projectionSize = numCols;
+    for (int i = 0; i < numCols; i++) {
+      projectedColumns[i] = i;
+    }
+  }
+
+  /**
+   * Returns the maximum size of the batch (number of rows it can hold)
+   */
+  public int getMaxSize() {
+      return selected.length;
+  }
+
+  /**
+   * Return count of qualifying rows.
+   *
+   * @return number of rows that have not been filtered out
+   */
+  public long count() {
+    return size;
+  }
+
+  private static String toUTF8(Object o) {
+    if(o == null || o instanceof NullWritable) {
+      return "\\N"; /* as found in LazySimpleSerDe's nullSequence */
+    }
+    return o.toString();
+  }
+
+  @Override
+  public String toString() {
+    if (size == 0) {
+      return "";
+    }
+    StringBuilder b = new StringBuilder();
+    if (this.selectedInUse) {
+      for (int j = 0; j < size; j++) {
+        int i = selected[j];
+        b.append('[');
+        for (int k = 0; k < projectionSize; k++) {
+          int projIndex = projectedColumns[k];
+          ColumnVector cv = cols[projIndex];
+          if (k > 0) {
+            b.append(", ");
+          }
+          cv.stringifyValue(b, i);
+        }
+        b.append(']');
+        if (j < size - 1) {
+          b.append('\n');
+        }
+      }
+    } else {
+      for (int i = 0; i < size; i++) {
+        b.append('[');
+        for (int k = 0; k < projectionSize; k++) {
+          int projIndex = projectedColumns[k];
+          ColumnVector cv = cols[projIndex];
+          if (k > 0) {
+            b.append(", ");
+          }
+          cv.stringifyValue(b, i);
+        }
+        b.append(']');
+        if (i < size - 1) {
+          b.append('\n');
+        }
+      }
+    }
+    return b.toString();
+  }
+
+  @Override
+  public void readFields(DataInput arg0) throws IOException {
+    throw new UnsupportedOperationException("Do you really need me?");
+  }
+
+  @Override
+  public void write(DataOutput arg0) throws IOException {
+    throw new UnsupportedOperationException("Don't call me");
+  }
+
+  /**
+   * Resets the row batch to default state
+   *  - sets selectedInUse to false
+   *  - sets size to 0
+   *  - sets endOfFile to false
+   *  - resets each column
+   *  - inits each column
+   */
+  public void reset() {
+    selectedInUse = false;
+    size = 0;
+    endOfFile = false;
+    for (ColumnVector vc : cols) {
+      if (vc != null) {
+        vc.reset();
+        vc.init();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/9ae70cb4/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/ExpressionTree.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/ExpressionTree.java b/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/ExpressionTree.java
new file mode 100644
index 0000000..577d95d
--- /dev/null
+++ b/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/ExpressionTree.java
@@ -0,0 +1,156 @@
+/**
+ * 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.io.sarg;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * The inner representation of the SearchArgument. Most users should not
+ * need this interface, it is only for file formats that need to translate
+ * the SearchArgument into an internal form.
+ */
+public class ExpressionTree {
+  public enum Operator {OR, AND, NOT, LEAF, CONSTANT}
+  private final Operator operator;
+  private final List<ExpressionTree> children;
+  private final int leaf;
+  private final SearchArgument.TruthValue constant;
+
+  ExpressionTree() {
+    operator = null;
+    children = null;
+    leaf = 0;
+    constant = null;
+  }
+
+  ExpressionTree(Operator op, ExpressionTree... kids) {
+    operator = op;
+    children = new ArrayList<ExpressionTree>();
+    leaf = -1;
+    this.constant = null;
+    Collections.addAll(children, kids);
+  }
+
+  ExpressionTree(int leaf) {
+    operator = Operator.LEAF;
+    children = null;
+    this.leaf = leaf;
+    this.constant = null;
+  }
+
+  ExpressionTree(SearchArgument.TruthValue constant) {
+    operator = Operator.CONSTANT;
+    children = null;
+    this.leaf = -1;
+    this.constant = constant;
+  }
+
+  ExpressionTree(ExpressionTree other) {
+    this.operator = other.operator;
+    if (other.children == null) {
+      this.children = null;
+    } else {
+      this.children = new ArrayList<ExpressionTree>();
+      for(ExpressionTree child: other.children) {
+        children.add(new ExpressionTree(child));
+      }
+    }
+    this.leaf = other.leaf;
+    this.constant = other.constant;
+  }
+
+  public SearchArgument.TruthValue evaluate(SearchArgument.TruthValue[] leaves
+                                            ) {
+    SearchArgument.TruthValue result = null;
+    switch (operator) {
+      case OR:
+        for(ExpressionTree child: children) {
+          result = child.evaluate(leaves).or(result);
+        }
+        return result;
+      case AND:
+        for(ExpressionTree child: children) {
+          result = child.evaluate(leaves).and(result);
+        }
+        return result;
+      case NOT:
+        return children.get(0).evaluate(leaves).not();
+      case LEAF:
+        return leaves[leaf];
+      case CONSTANT:
+        return constant;
+      default:
+        throw new IllegalStateException("Unknown operator: " + operator);
+    }
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder buffer = new StringBuilder();
+    switch (operator) {
+      case OR:
+        buffer.append("(or");
+        for(ExpressionTree child: children) {
+          buffer.append(' ');
+          buffer.append(child.toString());
+        }
+        buffer.append(')');
+        break;
+      case AND:
+        buffer.append("(and");
+        for(ExpressionTree child: children) {
+          buffer.append(' ');
+          buffer.append(child.toString());
+        }
+        buffer.append(')');
+        break;
+      case NOT:
+        buffer.append("(not ");
+        buffer.append(children.get(0));
+        buffer.append(')');
+        break;
+      case LEAF:
+        buffer.append("leaf-");
+        buffer.append(leaf);
+        break;
+      case CONSTANT:
+        buffer.append(constant);
+        break;
+    }
+    return buffer.toString();
+  }
+
+  public Operator getOperator() {
+    return operator;
+  }
+
+  public List<ExpressionTree> getChildren() {
+    return children;
+  }
+
+  public SearchArgument.TruthValue getConstant() {
+    return constant;
+  }
+
+  public int getLeaf() {
+    return leaf;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/9ae70cb4/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/PredicateLeaf.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/PredicateLeaf.java b/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/PredicateLeaf.java
new file mode 100644
index 0000000..3a92565
--- /dev/null
+++ b/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/PredicateLeaf.java
@@ -0,0 +1,104 @@
+/**
+ * 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.io.sarg;
+
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+
+/**
+ * The primitive predicates that form a SearchArgument.
+ */
+public interface PredicateLeaf {
+
+  /**
+   * The possible operators for predicates. To get the opposites, construct
+   * an expression with a not operator.
+   */
+  public static enum Operator {
+    EQUALS,
+    NULL_SAFE_EQUALS,
+    LESS_THAN,
+    LESS_THAN_EQUALS,
+    IN,
+    BETWEEN,
+    IS_NULL
+  }
+
+  /**
+   * The possible types for sargs.
+   */
+  public static enum Type {
+    INTEGER(Integer.class), // all of the integer types except long
+    LONG(Long.class),
+    FLOAT(Double.class),   // float and double
+    STRING(String.class),  // string, char, varchar
+    DATE(Date.class),
+    DECIMAL(HiveDecimalWritable.class),
+    TIMESTAMP(Timestamp.class),
+    BOOLEAN(Boolean.class);
+
+    private final Class cls;
+    Type(Class cls) {
+      this.cls = cls;
+    }
+
+    /**
+     * For all SARG leaves, the values must be the matching class.
+     * @return the value class
+     */
+    public Class getValueClass() {
+      return cls;
+    }
+  }
+
+  /**
+   * Get the operator for the leaf.
+   */
+  public Operator getOperator();
+
+  /**
+   * Get the type of the column and literal by the file format.
+   */
+  public Type getType();
+
+  /**
+   * Get the simple column name.
+   * @return the column name
+   */
+  public String getColumnName();
+
+  /**
+   * Get the literal half of the predicate leaf. Adapt the original type for what orc needs
+   *
+   * @return an Integer, Long, Double, or String
+   */
+  public Object getLiteral();
+
+  /**
+   * For operators with multiple literals (IN and BETWEEN), get the literals.
+   *
+   * @return the list of literals (Integer, Longs, Doubles, or Strings)
+   *
+   */
+  public List<Object> getLiteralList();
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/9ae70cb4/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgument.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgument.java b/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgument.java
new file mode 100644
index 0000000..d70b3b0
--- /dev/null
+++ b/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgument.java
@@ -0,0 +1,287 @@
+/**
+ * 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.io.sarg;
+
+import java.util.List;
+
+/**
+ * Primary interface for <a href="http://en.wikipedia.org/wiki/Sargable">
+ *   SearchArgument</a>, which are the subset of predicates
+ * that can be pushed down to the RecordReader. Each SearchArgument consists
+ * of a series of SearchClauses that must each be true for the row to be
+ * accepted by the filter.
+ *
+ * This requires that the filter be normalized into conjunctive normal form
+ * (<a href="http://en.wikipedia.org/wiki/Conjunctive_normal_form">CNF</a>).
+ */
+public interface SearchArgument {
+
+  /**
+   * The potential result sets of logical operations.
+   */
+  public static enum TruthValue {
+    YES, NO, NULL, YES_NULL, NO_NULL, YES_NO, YES_NO_NULL;
+
+    /**
+     * Compute logical or between the two values.
+     * @param right the other argument or null
+     * @return the result
+     */
+    public TruthValue or(TruthValue right) {
+      if (right == null || right == this) {
+        return this;
+      }
+      if (right == YES || this == YES) {
+        return YES;
+      }
+      if (right == YES_NULL || this == YES_NULL) {
+        return YES_NULL;
+      }
+      if (right == NO) {
+        return this;
+      }
+      if (this == NO) {
+        return right;
+      }
+      if (this == NULL) {
+        if (right == NO_NULL) {
+          return NULL;
+        } else {
+          return YES_NULL;
+        }
+      }
+      if (right == NULL) {
+        if (this == NO_NULL) {
+          return NULL;
+        } else {
+          return YES_NULL;
+        }
+      }
+      return YES_NO_NULL;
+    }
+
+    /**
+     * Compute logical AND between the two values.
+     * @param right the other argument or null
+     * @return the result
+     */
+    public TruthValue and(TruthValue right) {
+      if (right == null || right == this) {
+        return this;
+      }
+      if (right == NO || this == NO) {
+        return NO;
+      }
+      if (right == NO_NULL || this == NO_NULL) {
+        return NO_NULL;
+      }
+      if (right == YES) {
+        return this;
+      }
+      if (this == YES) {
+        return right;
+      }
+      if (this == NULL) {
+        if (right == YES_NULL) {
+          return NULL;
+        } else {
+          return NO_NULL;
+        }
+      }
+      if (right == NULL) {
+        if (this == YES_NULL) {
+          return NULL;
+        } else {
+          return NO_NULL;
+        }
+      }
+      return YES_NO_NULL;
+    }
+
+    public TruthValue not() {
+      switch (this) {
+        case NO:
+          return YES;
+        case YES:
+          return NO;
+        case NULL:
+        case YES_NO:
+        case YES_NO_NULL:
+          return this;
+        case NO_NULL:
+          return YES_NULL;
+        case YES_NULL:
+          return NO_NULL;
+        default:
+          throw new IllegalArgumentException("Unknown value: " + this);
+      }
+    }
+
+    /**
+     * Does the RecordReader need to include this set of records?
+     * @return true unless none of the rows qualify
+     */
+    public boolean isNeeded() {
+      switch (this) {
+        case NO:
+        case NULL:
+        case NO_NULL:
+          return false;
+        default:
+          return true;
+      }
+    }
+  }
+
+  /**
+   * Get the leaf predicates that are required to evaluate the predicate. The
+   * list will have the duplicates removed.
+   * @return the list of leaf predicates
+   */
+  public List<PredicateLeaf> getLeaves();
+
+  /**
+   * Get the expression tree. This should only needed for file formats that
+   * need to translate the expression to an internal form.
+   */
+  public ExpressionTree getExpression();
+
+  /**
+   * Evaluate the entire predicate based on the values for the leaf predicates.
+   * @param leaves the value of each leaf predicate
+   * @return the value of hte entire predicate
+   */
+  public TruthValue evaluate(TruthValue[] leaves);
+
+  /**
+   * A builder object for contexts outside of Hive where it isn't easy to
+   * get a ExprNodeDesc. The user must call startOr, startAnd, or startNot
+   * before adding any leaves.
+   */
+  public interface Builder {
+
+    /**
+     * Start building an or operation and push it on the stack.
+     * @return this
+     */
+    public Builder startOr();
+
+    /**
+     * Start building an and operation and push it on the stack.
+     * @return this
+     */
+    public Builder startAnd();
+
+    /**
+     * Start building a not operation and push it on the stack.
+     * @return this
+     */
+    public Builder startNot();
+
+    /**
+     * Finish the current operation and pop it off of the stack. Each start
+     * call must have a matching end.
+     * @return this
+     */
+    public Builder end();
+
+    /**
+     * Add a less than leaf to the current item on the stack.
+     * @param column the name of the column
+     * @param type the type of the expression
+     * @param literal the literal
+     * @return this
+     */
+    public Builder lessThan(String column, PredicateLeaf.Type type,
+                            Object literal);
+
+    /**
+     * Add a less than equals leaf to the current item on the stack.
+     * @param column the name of the column
+     * @param type the type of the expression
+     * @param literal the literal
+     * @return this
+     */
+    public Builder lessThanEquals(String column, PredicateLeaf.Type type,
+                                  Object literal);
+
+    /**
+     * Add an equals leaf to the current item on the stack.
+     * @param column the name of the column
+     * @param type the type of the expression
+     * @param literal the literal
+     * @return this
+     */
+    public Builder equals(String column, PredicateLeaf.Type type,
+                          Object literal);
+
+    /**
+     * Add a null safe equals leaf to the current item on the stack.
+     * @param column the name of the column
+     * @param type the type of the expression
+     * @param literal the literal
+     * @return this
+     */
+    public Builder nullSafeEquals(String column, PredicateLeaf.Type type,
+                                  Object literal);
+
+    /**
+     * Add an in leaf to the current item on the stack.
+     * @param column the name of the column
+     * @param type the type of the expression
+     * @param literal the literal
+     * @return this
+     */
+    public Builder in(String column, PredicateLeaf.Type type,
+                      Object... literal);
+
+    /**
+     * Add an is null leaf to the current item on the stack.
+     * @param column the name of the column
+     * @param type the type of the expression
+     * @return this
+     */
+    public Builder isNull(String column, PredicateLeaf.Type type);
+
+    /**
+     * Add a between leaf to the current item on the stack.
+     * @param column the name of the column
+     * @param type the type of the expression
+     * @param lower the literal
+     * @param upper the literal
+     * @return this
+     */
+    public Builder between(String column, PredicateLeaf.Type type,
+                           Object lower, Object upper);
+
+    /**
+     * Add a truth value to the expression.
+     * @param truth
+     * @return this
+     */
+    public Builder literal(TruthValue truth);
+
+    /**
+     * Build and return the SearchArgument that has been defined. All of the
+     * starts must have been ended before this call.
+     * @return the new SearchArgument
+     */
+    public SearchArgument build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/9ae70cb4/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentFactory.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentFactory.java b/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentFactory.java
new file mode 100644
index 0000000..0778935
--- /dev/null
+++ b/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentFactory.java
@@ -0,0 +1,28 @@
+/**
+ * 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.io.sarg;
+
+/**
+ * A factory for creating SearchArguments.
+ */
+public class SearchArgumentFactory {
+  public static SearchArgument.Builder newBuilder() {
+    return new SearchArgumentImpl.BuilderImpl();
+  }
+}


[04/43] hive git commit: HIVE-11311: Avoid dumping AST tree String in Explain unless necessary (Jesus Camacho Rodriguez, reviewed by Hari Sankar Sivarama Subramaniyan)

Posted by xu...@apache.org.
HIVE-11311: Avoid dumping AST tree String in Explain unless necessary (Jesus Camacho Rodriguez, reviewed by Hari Sankar Sivarama Subramaniyan)


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

Branch: refs/heads/spark
Commit: dfdc6700c7a81320a3e0909305645db0248bf01a
Parents: 55853b4
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Tue Jul 21 17:38:39 2015 +0100
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Tue Jul 21 17:38:39 2015 +0100

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hive/ql/Driver.java  |  6 ++--
 .../apache/hadoop/hive/ql/exec/ExplainTask.java |  7 ++--
 .../hive/ql/parse/ExplainSemanticAnalyzer.java  |  2 +-
 .../apache/hadoop/hive/ql/plan/ExplainWork.java | 19 +++++++----
 .../parse/TestUpdateDeleteSemanticAnalyzer.java | 34 ++++++++++----------
 5 files changed, 37 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/dfdc6700/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
index b74e5fa..6ad3f49 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
@@ -482,7 +482,7 @@ public class Driver implements CommandProcessor {
       }
 
       if (conf.getBoolVar(ConfVars.HIVE_LOG_EXPLAIN_OUTPUT)) {
-        String explainOutput = getExplainOutput(sem, plan, tree.dump());
+        String explainOutput = getExplainOutput(sem, plan, tree);
         if (explainOutput != null) {
           LOG.info("EXPLAIN output for queryid " + queryId + " : "
               + explainOutput);
@@ -533,7 +533,7 @@ public class Driver implements CommandProcessor {
    * @throws java.io.IOException
    */
   private String getExplainOutput(BaseSemanticAnalyzer sem, QueryPlan plan,
-      String astStringTree) throws IOException {
+      ASTNode astTree) throws IOException {
     String ret = null;
     ExplainTask task = new ExplainTask();
     task.initialize(conf, plan, null);
@@ -541,7 +541,7 @@ public class Driver implements CommandProcessor {
     PrintStream ps = new PrintStream(baos);
     try {
       List<Task<?>> rootTasks = sem.getRootTasks();
-      task.getJSONPlan(ps, astStringTree, rootTasks, sem.getFetchTask(), false, true, true);
+      task.getJSONPlan(ps, astTree, rootTasks, sem.getFetchTask(), false, true, true);
       ret = baos.toString();
     } catch (Exception e) {
       LOG.warn("Exception generating explain output: " + e, e);

http://git-wip-us.apache.org/repos/asf/hive/blob/dfdc6700/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java
index c6b49bf..21de3cf 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java
@@ -50,6 +50,7 @@ import org.apache.hadoop.hive.ql.DriverContext;
 import org.apache.hadoop.hive.ql.hooks.ReadEntity;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.optimizer.physical.StageIDsRearranger;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
 import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer;
 import org.apache.hadoop.hive.ql.plan.Explain;
 import org.apache.hadoop.hive.ql.plan.Explain.Level;
@@ -171,11 +172,11 @@ public class ExplainTask extends Task<ExplainWork> implements Serializable {
 
   public JSONObject getJSONPlan(PrintStream out, ExplainWork work)
       throws Exception {
-    return getJSONPlan(out, work.getAstStringTree(), work.getRootTasks(), work.getFetchTask(),
+    return getJSONPlan(out, work.getAstTree(), work.getRootTasks(), work.getFetchTask(),
                        work.isFormatted(), work.getExtended(), work.isAppendTaskType());
   }
 
-  public JSONObject getJSONPlan(PrintStream out, String ast, List<Task<?>> tasks, Task<?> fetchTask,
+  public JSONObject getJSONPlan(PrintStream out, ASTNode ast, List<Task<?>> tasks, Task<?> fetchTask,
       boolean jsonOutput, boolean isExtended, boolean appendTaskType) throws Exception {
 
     // If the user asked for a formatted output, dump the json output
@@ -188,7 +189,7 @@ public class ExplainTask extends Task<ExplainWork> implements Serializable {
 
     // Print out the parse AST
     if (ast != null && isExtended) {
-      String jsonAST = outputAST(ast, out, jsonOutput, 0);
+      String jsonAST = outputAST(ast.dump(), out, jsonOutput, 0);
       if (out != null) {
         out.println();
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/dfdc6700/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java
index 66d1546..2d365a9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java
@@ -102,7 +102,7 @@ public class ExplainSemanticAnalyzer extends BaseSemanticAnalyzer {
         pCtx,
         tasks,
         fetchTask,
-        input.dump(),
+        input,
         sem,
         extended,
         formatted,

http://git-wip-us.apache.org/repos/asf/hive/blob/dfdc6700/ql/src/java/org/apache/hadoop/hive/ql/plan/ExplainWork.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/ExplainWork.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/ExplainWork.java
index aa208a5..7a2f883 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/ExplainWork.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/ExplainWork.java
@@ -26,8 +26,9 @@ import java.util.List;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.hooks.ReadEntity;
-import org.apache.hadoop.hive.ql.parse.ParseContext;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
 import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer;
+import org.apache.hadoop.hive.ql.parse.ParseContext;
 
 /**
  * ExplainWork.
@@ -39,6 +40,7 @@ public class ExplainWork implements Serializable {
   private Path resFile;
   private ArrayList<Task<? extends Serializable>> rootTasks;
   private Task<? extends Serializable> fetchTask;
+  private ASTNode astTree;
   private String astStringTree;
   private HashSet<ReadEntity> inputs;
   private ParseContext pCtx;
@@ -63,7 +65,7 @@ public class ExplainWork implements Serializable {
       ParseContext pCtx,
       List<Task<? extends Serializable>> rootTasks,
       Task<? extends Serializable> fetchTask,
-      String astStringTree,
+      ASTNode astTree,
       BaseSemanticAnalyzer analyzer,
       boolean extended,
       boolean formatted,
@@ -75,7 +77,7 @@ public class ExplainWork implements Serializable {
     this.resFile = resFile;
     this.rootTasks = new ArrayList<Task<? extends Serializable>>(rootTasks);
     this.fetchTask = fetchTask;
-    this.astStringTree = astStringTree;
+    this.astTree = astTree;
     this.analyzer = analyzer;
     this.inputs = analyzer.getInputs();
     this.extended = extended;
@@ -112,12 +114,15 @@ public class ExplainWork implements Serializable {
     this.fetchTask = fetchTask;
   }
 
-  public String getAstStringTree() {
-    return astStringTree;
+  public ASTNode getAstTree() {
+    return astTree;
   }
 
-  public void setAstStringTree(String astStringTree) {
-    this.astStringTree = astStringTree;
+  public String getAstStringTree() {
+    if (astStringTree == null) {
+      astStringTree = astTree.dump();
+    }
+    return astStringTree;
   }
 
   public HashSet<ReadEntity> getInputs() {

http://git-wip-us.apache.org/repos/asf/hive/blob/dfdc6700/ql/src/test/org/apache/hadoop/hive/ql/parse/TestUpdateDeleteSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestUpdateDeleteSemanticAnalyzer.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestUpdateDeleteSemanticAnalyzer.java
index f0435cb..f78f226 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestUpdateDeleteSemanticAnalyzer.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestUpdateDeleteSemanticAnalyzer.java
@@ -59,7 +59,7 @@ public class TestUpdateDeleteSemanticAnalyzer {
     try {
       ReturnInfo rc = parseAndAnalyze("insert into table T select a, b from U", "testInsertSelect");
 
-      LOG.info(explain((SemanticAnalyzer)rc.sem, rc.plan, rc.ast.dump()));
+      LOG.info(explain((SemanticAnalyzer)rc.sem, rc.plan, rc.ast));
 
     } finally {
       cleanupTables();
@@ -70,7 +70,7 @@ public class TestUpdateDeleteSemanticAnalyzer {
   public void testDeleteAllNonPartitioned() throws Exception {
     try {
       ReturnInfo rc = parseAndAnalyze("delete from T", "testDeleteAllNonPartitioned");
-      LOG.info(explain((SemanticAnalyzer)rc.sem, rc.plan, rc.ast.dump()));
+      LOG.info(explain((SemanticAnalyzer)rc.sem, rc.plan, rc.ast));
     } finally {
       cleanupTables();
     }
@@ -80,7 +80,7 @@ public class TestUpdateDeleteSemanticAnalyzer {
   public void testDeleteWhereNoPartition() throws Exception {
     try {
       ReturnInfo rc = parseAndAnalyze("delete from T where a > 5", "testDeleteWhereNoPartition");
-      LOG.info(explain((SemanticAnalyzer)rc.sem, rc.plan, rc.ast.dump()));
+      LOG.info(explain((SemanticAnalyzer)rc.sem, rc.plan, rc.ast));
     } finally {
       cleanupTables();
     }
@@ -90,7 +90,7 @@ public class TestUpdateDeleteSemanticAnalyzer {
   public void testDeleteAllPartitioned() throws Exception {
     try {
       ReturnInfo rc = parseAndAnalyze("delete from U", "testDeleteAllPartitioned");
-      LOG.info(explain((SemanticAnalyzer)rc.sem, rc.plan, rc.ast.dump()));
+      LOG.info(explain((SemanticAnalyzer)rc.sem, rc.plan, rc.ast));
     } finally {
       cleanupTables();
     }
@@ -100,7 +100,7 @@ public class TestUpdateDeleteSemanticAnalyzer {
   public void testDeleteAllWherePartitioned() throws Exception {
     try {
       ReturnInfo rc = parseAndAnalyze("delete from U where a > 5", "testDeleteAllWherePartitioned");
-      LOG.info(explain((SemanticAnalyzer)rc.sem, rc.plan, rc.ast.dump()));
+      LOG.info(explain((SemanticAnalyzer)rc.sem, rc.plan, rc.ast));
     } finally {
       cleanupTables();
     }
@@ -111,7 +111,7 @@ public class TestUpdateDeleteSemanticAnalyzer {
     try {
       ReturnInfo rc = parseAndAnalyze("delete from U where ds = 'today'",
           "testDeleteFromPartitionOnly");
-      LOG.info(explain((SemanticAnalyzer)rc.sem, rc.plan, rc.ast.dump()));
+      LOG.info(explain((SemanticAnalyzer)rc.sem, rc.plan, rc.ast));
     } finally {
       cleanupTables();
     }
@@ -122,7 +122,7 @@ public class TestUpdateDeleteSemanticAnalyzer {
     try {
       ReturnInfo rc = parseAndAnalyze("delete from U where ds = 'today' and a > 5",
           "testDeletePartitionWhere");
-      LOG.info(explain((SemanticAnalyzer)rc.sem, rc.plan, rc.ast.dump()));
+      LOG.info(explain((SemanticAnalyzer)rc.sem, rc.plan, rc.ast));
     } finally {
       cleanupTables();
     }
@@ -132,7 +132,7 @@ public class TestUpdateDeleteSemanticAnalyzer {
   public void testUpdateAllNonPartitioned() throws Exception {
     try {
       ReturnInfo rc = parseAndAnalyze("update T set b = 5", "testUpdateAllNonPartitioned");
-      LOG.info(explain((SemanticAnalyzer)rc.sem, rc.plan, rc.ast.dump()));
+      LOG.info(explain((SemanticAnalyzer)rc.sem, rc.plan, rc.ast));
     } finally {
       cleanupTables();
     }
@@ -143,7 +143,7 @@ public class TestUpdateDeleteSemanticAnalyzer {
     try {
       ReturnInfo rc = parseAndAnalyze("update T set b = 5 where b > 5",
           "testUpdateAllNonPartitionedWhere");
-      LOG.info(explain((SemanticAnalyzer)rc.sem, rc.plan, rc.ast.dump()));
+      LOG.info(explain((SemanticAnalyzer)rc.sem, rc.plan, rc.ast));
     } finally {
       cleanupTables();
     }
@@ -153,7 +153,7 @@ public class TestUpdateDeleteSemanticAnalyzer {
   public void testUpdateAllPartitioned() throws Exception {
     try {
       ReturnInfo rc = parseAndAnalyze("update U set b = 5", "testUpdateAllPartitioned");
-      LOG.info(explain((SemanticAnalyzer)rc.sem, rc.plan, rc.ast.dump()));
+      LOG.info(explain((SemanticAnalyzer)rc.sem, rc.plan, rc.ast));
     } finally {
       cleanupTables();
     }
@@ -164,7 +164,7 @@ public class TestUpdateDeleteSemanticAnalyzer {
     try {
       ReturnInfo rc = parseAndAnalyze("update U set b = 5 where b > 5",
           "testUpdateAllPartitionedWhere");
-      LOG.info(explain((SemanticAnalyzer)rc.sem, rc.plan, rc.ast.dump()));
+      LOG.info(explain((SemanticAnalyzer)rc.sem, rc.plan, rc.ast));
     } finally {
       cleanupTables();
     }
@@ -175,7 +175,7 @@ public class TestUpdateDeleteSemanticAnalyzer {
     try {
       ReturnInfo rc = parseAndAnalyze("update U set b = 5 where ds = 'today'",
           "testUpdateOnePartition");
-      LOG.info(explain((SemanticAnalyzer)rc.sem, rc.plan, rc.ast.dump()));
+      LOG.info(explain((SemanticAnalyzer)rc.sem, rc.plan, rc.ast));
     } finally {
       cleanupTables();
     }
@@ -186,7 +186,7 @@ public class TestUpdateDeleteSemanticAnalyzer {
     try {
       ReturnInfo rc = parseAndAnalyze("update U set b = 5 where ds = 'today' and b > 5",
           "testUpdateOnePartitionWhere");
-      LOG.info(explain((SemanticAnalyzer)rc.sem, rc.plan, rc.ast.dump()));
+      LOG.info(explain((SemanticAnalyzer)rc.sem, rc.plan, rc.ast));
     } finally {
       cleanupTables();
     }
@@ -198,7 +198,7 @@ public class TestUpdateDeleteSemanticAnalyzer {
       ReturnInfo rc = parseAndAnalyze("insert into table T values ('abc', 3), ('ghi', null)",
           "testInsertValues");
 
-      LOG.info(explain((SemanticAnalyzer)rc.sem, rc.plan, rc.ast.dump()));
+      LOG.info(explain((SemanticAnalyzer)rc.sem, rc.plan, rc.ast));
 
     } finally {
       cleanupTables();
@@ -212,7 +212,7 @@ public class TestUpdateDeleteSemanticAnalyzer {
               "('abc', 3, 'today'), ('ghi', 5, 'tomorrow')",
           "testInsertValuesPartitioned");
 
-      LOG.info(explain((SemanticAnalyzer) rc.sem, rc.plan, rc.ast.dump()));
+      LOG.info(explain((SemanticAnalyzer) rc.sem, rc.plan, rc.ast));
 
     } finally {
       cleanupTables();
@@ -285,7 +285,7 @@ public class TestUpdateDeleteSemanticAnalyzer {
     return new ReturnInfo(tree, sem, plan);
   }
 
-  private String explain(SemanticAnalyzer sem, QueryPlan plan, String astStringTree) throws
+  private String explain(SemanticAnalyzer sem, QueryPlan plan, ASTNode astTree) throws
       IOException {
     FileSystem fs = FileSystem.get(conf);
     File f = File.createTempFile("TestSemanticAnalyzer", "explain");
@@ -293,7 +293,7 @@ public class TestUpdateDeleteSemanticAnalyzer {
     fs.create(tmp);
     fs.deleteOnExit(tmp);
     ExplainWork work = new ExplainWork(tmp, sem.getParseContext(), sem.getRootTasks(),
-        sem.getFetchTask(), astStringTree, sem, true, false, false, false, false, false, null);
+        sem.getFetchTask(), astTree, sem, true, false, false, false, false, false, null);
     ExplainTask task = new ExplainTask();
     task.setWork(work);
     task.initialize(conf, plan, null);


[27/43] hive git commit: HIVE-10799. Refactor the SearchArgumentFactory to remove the AST-specific factory. (omalley reviewed by prasanth_j)

Posted by xu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/c178a6e9/ql/src/test/org/apache/hadoop/hive/ql/io/sarg/TestSearchArgumentImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/sarg/TestSearchArgumentImpl.java b/ql/src/test/org/apache/hadoop/hive/ql/io/sarg/TestSearchArgumentImpl.java
index 46ce49c..3a51f4a 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/sarg/TestSearchArgumentImpl.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/sarg/TestSearchArgumentImpl.java
@@ -19,27 +19,23 @@
 package org.apache.hadoop.hive.ql.io.sarg;
 
 import static junit.framework.Assert.assertEquals;
-import static junit.framework.Assert.assertNull;
 import static junit.framework.Assert.assertTrue;
 
 import com.google.common.collect.Sets;
 
 import org.apache.hadoop.hive.common.type.HiveChar;
-import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.common.type.HiveVarchar;
-import org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument.TruthValue;
-import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentImpl.ExpressionBuilder;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentImpl.PredicateLeafImpl;
 import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
-import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.junit.Test;
 
 import java.beans.XMLDecoder;
 import java.io.ByteArrayInputStream;
 import java.io.UnsupportedEncodingException;
 import java.lang.reflect.Field;
-import java.math.BigDecimal;
+import java.sql.Date;
 import java.sql.Timestamp;
 import java.util.List;
 import java.util.Set;
@@ -92,96 +88,103 @@ public class TestSearchArgumentImpl {
 
   @Test
   public void testNotPushdown() throws Exception {
-    assertEquals("leaf-1", ExpressionBuilder.pushDownNot(leaf(1)).toString());
+    assertEquals("leaf-1", SearchArgumentImpl.BuilderImpl.pushDownNot(leaf(1))
+        .toString());
     assertEquals("(not leaf-1)",
-        ExpressionBuilder.pushDownNot(not(leaf(1))).toString());
+        SearchArgumentImpl.BuilderImpl.pushDownNot(not(leaf(1))).toString());
     assertEquals("leaf-1",
-        ExpressionBuilder.pushDownNot(not(not(leaf(1)))).toString());
+        SearchArgumentImpl.BuilderImpl.pushDownNot(not(not(leaf(1))))
+            .toString());
     assertEquals("(not leaf-1)",
-        ExpressionBuilder.pushDownNot(not(not(not(leaf(1))))).toString());
+        SearchArgumentImpl.BuilderImpl.pushDownNot(not(not(not(leaf(1))))).
+            toString());
     assertEquals("(or leaf-1 (not leaf-2))",
-        ExpressionBuilder.pushDownNot(not(and(not(leaf(1)),
+        SearchArgumentImpl.BuilderImpl.pushDownNot(not(and(not(leaf(1)),
             leaf(2)))).toString());
     assertEquals("(and (not leaf-1) leaf-2)",
-        ExpressionBuilder.pushDownNot(not(or(leaf(1),
+        SearchArgumentImpl.BuilderImpl.pushDownNot(not(or(leaf(1),
             not(leaf(2))))).toString());
     assertEquals("(or (or (not leaf-1) leaf-2) leaf-3)",
-        ExpressionBuilder.pushDownNot(or(not(and(leaf(1), not(leaf(2)))),
+        SearchArgumentImpl.BuilderImpl.pushDownNot(or(not(and(leaf(1),
+                not(leaf(2)))),
             not(not(leaf(3))))).toString());
-    assertEquals("NO", ExpressionBuilder.pushDownNot(
+    assertEquals("NO", SearchArgumentImpl.BuilderImpl.pushDownNot(
         not(constant(TruthValue.YES))).toString());
-    assertEquals("YES", ExpressionBuilder.pushDownNot(
+    assertEquals("YES", SearchArgumentImpl.BuilderImpl.pushDownNot(
         not(constant(TruthValue.NO))).toString());
-    assertEquals("NULL", ExpressionBuilder.pushDownNot(
+    assertEquals("NULL", SearchArgumentImpl.BuilderImpl.pushDownNot(
         not(constant(TruthValue.NULL))).toString());
-    assertEquals("YES_NO", ExpressionBuilder.pushDownNot(
+    assertEquals("YES_NO", SearchArgumentImpl.BuilderImpl.pushDownNot(
         not(constant(TruthValue.YES_NO))).toString());
-    assertEquals("YES_NULL", ExpressionBuilder.pushDownNot(
+    assertEquals("YES_NULL", SearchArgumentImpl.BuilderImpl.pushDownNot(
         not(constant(TruthValue.NO_NULL))).toString());
-    assertEquals("NO_NULL", ExpressionBuilder.pushDownNot(
+    assertEquals("NO_NULL", SearchArgumentImpl.BuilderImpl.pushDownNot(
         not(constant(TruthValue.YES_NULL))).toString());
-    assertEquals("YES_NO_NULL", ExpressionBuilder.pushDownNot(
+    assertEquals("YES_NO_NULL", SearchArgumentImpl.BuilderImpl.pushDownNot(
         not(constant(TruthValue.YES_NO_NULL))).toString());
   }
 
   @Test
   public void testFlatten() throws Exception {
-    assertEquals("leaf-1", ExpressionBuilder.flatten(leaf(1)).toString());
+    assertEquals("leaf-1", SearchArgumentImpl.BuilderImpl.flatten(leaf(1)).toString());
     assertEquals("NO",
-        ExpressionBuilder.flatten(constant(TruthValue.NO)).toString());
+        SearchArgumentImpl.BuilderImpl.flatten(constant(TruthValue.NO)).toString());
     assertEquals("(not (not leaf-1))",
-        ExpressionBuilder.flatten(not(not(leaf(1)))).toString());
+        SearchArgumentImpl.BuilderImpl.flatten(not(not(leaf(1)))).toString());
     assertEquals("(and leaf-1 leaf-2)",
-        ExpressionBuilder.flatten(and(leaf(1), leaf(2))).toString());
+        SearchArgumentImpl.BuilderImpl.flatten(and(leaf(1), leaf(2))).toString());
     assertEquals("(and (or leaf-1 leaf-2) leaf-3)",
-        ExpressionBuilder.flatten(and(or(leaf(1), leaf(2)), leaf(3))
+        SearchArgumentImpl.BuilderImpl.flatten(and(or(leaf(1), leaf(2)), leaf(3))
         ).toString());
     assertEquals("(and leaf-1 leaf-2 leaf-3 leaf-4)",
-        ExpressionBuilder.flatten(and(and(leaf(1), leaf(2)),
+        SearchArgumentImpl.BuilderImpl.flatten(and(and(leaf(1), leaf(2)),
             and(leaf(3), leaf(4)))).toString());
     assertEquals("(or leaf-1 leaf-2 leaf-3 leaf-4)",
-        ExpressionBuilder.flatten(or(leaf(1), or(leaf(2), or(leaf(3),
+        SearchArgumentImpl.BuilderImpl.flatten(or(leaf(1), or(leaf(2), or(leaf(3),
             leaf(4))))).toString());
     assertEquals("(or leaf-1 leaf-2 leaf-3 leaf-4)",
-        ExpressionBuilder.flatten(or(or(or(leaf(1), leaf(2)), leaf(3)),
+        SearchArgumentImpl.BuilderImpl.flatten(or(or(or(leaf(1), leaf(2)), leaf(3)),
             leaf(4))).toString());
     assertEquals("(or leaf-1 leaf-2 leaf-3 leaf-4 leaf-5 leaf-6)",
-        ExpressionBuilder.flatten(or(or(leaf(1), or(leaf(2), leaf(3))),
+        SearchArgumentImpl.BuilderImpl.flatten(or(or(leaf(1), or(leaf(2), leaf(3))),
             or(or(leaf(4), leaf(5)), leaf(6)))).toString());
     assertEquals("(and (not leaf-1) leaf-2 (not leaf-3) leaf-4 (not leaf-5) leaf-6)",
-        ExpressionBuilder.flatten(and(and(not(leaf(1)), and(leaf(2),
-            not(leaf(3)))), and(and(leaf(4), not(leaf(5))), leaf(6)))
+        SearchArgumentImpl.BuilderImpl.flatten(and(and(not(leaf(1)), and(leaf(2),
+                not(leaf(3)))), and(and(leaf(4), not(leaf(5))), leaf(6)))
         ).toString());
     assertEquals("(not (and leaf-1 leaf-2 leaf-3))",
-        ExpressionBuilder.flatten(not(and(leaf(1), and(leaf(2), leaf(3))))
+        SearchArgumentImpl.BuilderImpl.flatten(not(and(leaf(1), and(leaf(2), leaf(3))))
         ).toString());
   }
 
   @Test
   public void testFoldMaybe() throws Exception {
     assertEquals("(and leaf-1)",
-        ExpressionBuilder.foldMaybe(and(leaf(1),
+        SearchArgumentImpl.BuilderImpl.foldMaybe(and(leaf(1),
             constant(TruthValue.YES_NO_NULL))).toString());
     assertEquals("(and leaf-1 leaf-2)",
-        ExpressionBuilder.foldMaybe(and(leaf(1),
+        SearchArgumentImpl.BuilderImpl.foldMaybe(and(leaf(1),
             constant(TruthValue.YES_NO_NULL), leaf(2))).toString());
     assertEquals("(and leaf-1 leaf-2)",
-        ExpressionBuilder.foldMaybe(and(constant(TruthValue.YES_NO_NULL),
-            leaf(1), leaf(2), constant(TruthValue.YES_NO_NULL))).toString());
+        SearchArgumentImpl.BuilderImpl.
+            foldMaybe(and(constant(TruthValue.YES_NO_NULL),
+                leaf(1), leaf(2), constant(TruthValue.YES_NO_NULL))).toString());
     assertEquals("YES_NO_NULL",
-        ExpressionBuilder.foldMaybe(and(constant(TruthValue.YES_NO_NULL),
-            constant(TruthValue.YES_NO_NULL))).toString());
+        SearchArgumentImpl.BuilderImpl.
+            foldMaybe(and(constant(TruthValue.YES_NO_NULL),
+                constant(TruthValue.YES_NO_NULL))).toString());
     assertEquals("YES_NO_NULL",
-        ExpressionBuilder.foldMaybe(or(leaf(1),
-            constant(TruthValue.YES_NO_NULL))).toString());
+        SearchArgumentImpl.BuilderImpl.
+            foldMaybe(or(leaf(1),
+                constant(TruthValue.YES_NO_NULL))).toString());
     assertEquals("(or leaf-1 (and leaf-2))",
-        ExpressionBuilder.foldMaybe(or(leaf(1),
+        SearchArgumentImpl.BuilderImpl.foldMaybe(or(leaf(1),
             and(leaf(2), constant(TruthValue.YES_NO_NULL)))).toString());
     assertEquals("(and leaf-1)",
-        ExpressionBuilder.foldMaybe(and(or(leaf(2),
+        SearchArgumentImpl.BuilderImpl.foldMaybe(and(or(leaf(2),
             constant(TruthValue.YES_NO_NULL)), leaf(1))).toString());
-    assertEquals("(and leaf-100)", ExpressionBuilder.foldMaybe(
-        ExpressionBuilder.convertToCNF(and(leaf(100),
+    assertEquals("(and leaf-100)", SearchArgumentImpl.BuilderImpl.foldMaybe(
+        SearchArgumentImpl.BuilderImpl.convertToCNF(and(leaf(100),
             or(and(leaf(0), leaf(1)),
                 and(leaf(2), leaf(3)),
                 and(leaf(4), leaf(5)),
@@ -195,28 +198,31 @@ public class TestSearchArgumentImpl {
 
   @Test
   public void testCNF() throws Exception {
-    assertEquals("leaf-1", ExpressionBuilder.convertToCNF(leaf(1)).toString());
-    assertEquals("NO", ExpressionBuilder.convertToCNF(
+    assertEquals("leaf-1", SearchArgumentImpl.BuilderImpl.convertToCNF(leaf(1)).
+        toString());
+    assertEquals("NO", SearchArgumentImpl.BuilderImpl.convertToCNF(
         constant(TruthValue.NO)).toString());
-    assertEquals("(not leaf-1)", ExpressionBuilder.convertToCNF(
+    assertEquals("(not leaf-1)", SearchArgumentImpl.BuilderImpl.convertToCNF(
         not(leaf(1))).toString());
-    assertEquals("(and leaf-1 leaf-2)", ExpressionBuilder.convertToCNF(
-        and(leaf(1), leaf(2))).toString());
-    assertEquals("(or (not leaf-1) leaf-2)", ExpressionBuilder.convertToCNF(
-        or(not(leaf(1)), leaf(2))).toString());
+    assertEquals("(and leaf-1 leaf-2)", SearchArgumentImpl.BuilderImpl.
+        convertToCNF(
+            and(leaf(1), leaf(2))).toString());
+    assertEquals("(or (not leaf-1) leaf-2)", SearchArgumentImpl.BuilderImpl.
+        convertToCNF(
+            or(not(leaf(1)), leaf(2))).toString());
     assertEquals("(and (or leaf-1 leaf-2) (not leaf-3))",
-        ExpressionBuilder.convertToCNF(
+        SearchArgumentImpl.BuilderImpl.convertToCNF(
             and(or(leaf(1), leaf(2)), not(leaf(3)))).toString());
     assertEquals("(and (or leaf-1 leaf-3) (or leaf-2 leaf-3)" +
         " (or leaf-1 leaf-4) (or leaf-2 leaf-4))",
-        ExpressionBuilder.convertToCNF(
+        SearchArgumentImpl.BuilderImpl.convertToCNF(
             or(and(leaf(1), leaf(2)), and(leaf(3), leaf(4)))).toString());
     assertEquals("(and" +
         " (or leaf-1 leaf-5) (or leaf-2 leaf-5)" +
         " (or leaf-3 leaf-5) (or leaf-4 leaf-5)" +
         " (or leaf-1 leaf-6) (or leaf-2 leaf-6)" +
         " (or leaf-3 leaf-6) (or leaf-4 leaf-6))",
-        ExpressionBuilder.convertToCNF(
+        SearchArgumentImpl.BuilderImpl.convertToCNF(
             or(and(leaf(1), leaf(2), leaf(3), leaf(4)),
                 and(leaf(5), leaf(6)))).toString());
     assertEquals("(and" +
@@ -224,7 +230,7 @@ public class TestSearchArgumentImpl {
         " (or leaf-5 leaf-6 (not leaf-7) leaf-2 leaf-3)" +
         " (or leaf-5 leaf-6 (not leaf-7) leaf-1 leaf-4)" +
         " (or leaf-5 leaf-6 (not leaf-7) leaf-2 leaf-4))",
-        ExpressionBuilder.convertToCNF(
+        SearchArgumentImpl.BuilderImpl.convertToCNF(
             or(and(leaf(1), leaf(2)),
                 and(leaf(3), leaf(4)),
                 or(leaf(5), leaf(6)),
@@ -248,33 +254,37 @@ public class TestSearchArgumentImpl {
         " (or leaf-8 leaf-0 leaf-5 leaf-7)" +
         " (or leaf-8 leaf-1 leaf-5 leaf-7)" +
         " (or leaf-8 leaf-2 leaf-5 leaf-7))",
-        ExpressionBuilder.convertToCNF(or(and(leaf(0), leaf(1), leaf(2)),
+        SearchArgumentImpl.BuilderImpl.convertToCNF(or(and(leaf(0), leaf(1),
+                leaf(2)),
             and(leaf(3), leaf(4), leaf(5)),
             and(leaf(6), leaf(7)),
             leaf(8))).toString());
-    assertEquals("YES_NO_NULL", ExpressionBuilder.convertToCNF(or(and(leaf(0), leaf(1)),
-        and(leaf(2), leaf(3)),
-        and(leaf(4), leaf(5)),
-        and(leaf(6), leaf(7)),
-        and(leaf(8), leaf(9)),
-        and(leaf(10), leaf(11)),
-        and(leaf(12), leaf(13)),
-        and(leaf(14), leaf(15)),
-        and(leaf(16), leaf(17)))).toString());
-    assertEquals("(and leaf-100 YES_NO_NULL)", ExpressionBuilder.convertToCNF(and(leaf(100),
-        or(and(leaf(0), leaf(1)),
-        and(leaf(2), leaf(3)),
-        and(leaf(4), leaf(5)),
-        and(leaf(6), leaf(7)),
-        and(leaf(8), leaf(9)),
-        and(leaf(10), leaf(11)),
-        and(leaf(12), leaf(13)),
-        and(leaf(14), leaf(15)),
-        and(leaf(16), leaf(17))))).toString());
-    assertNoSharedNodes(ExpressionBuilder.convertToCNF(or(and(leaf(0), leaf(1), leaf(2)),
-        and(leaf(3), leaf(4), leaf(5)),
-        and(leaf(6), leaf(7)),
-        leaf(8))), Sets.<ExpressionTree>newIdentityHashSet());
+    assertEquals("YES_NO_NULL", SearchArgumentImpl.BuilderImpl.
+        convertToCNF(or(and(leaf(0), leaf(1)),
+            and(leaf(2), leaf(3)),
+            and(leaf(4), leaf(5)),
+            and(leaf(6), leaf(7)),
+            and(leaf(8), leaf(9)),
+            and(leaf(10), leaf(11)),
+            and(leaf(12), leaf(13)),
+            and(leaf(14), leaf(15)),
+            and(leaf(16), leaf(17)))).toString());
+    assertEquals("(and leaf-100 YES_NO_NULL)", SearchArgumentImpl.BuilderImpl.
+        convertToCNF(and(leaf(100),
+            or(and(leaf(0), leaf(1)),
+                and(leaf(2), leaf(3)),
+                and(leaf(4), leaf(5)),
+                and(leaf(6), leaf(7)),
+                and(leaf(8), leaf(9)),
+                and(leaf(10), leaf(11)),
+                and(leaf(12), leaf(13)),
+                and(leaf(14), leaf(15)),
+                and(leaf(16), leaf(17))))).toString());
+    assertNoSharedNodes(SearchArgumentImpl.BuilderImpl.
+        convertToCNF(or(and(leaf(0), leaf(1), leaf(2)),
+            and(leaf(3), leaf(4), leaf(5)),
+            and(leaf(6), leaf(7)),
+            leaf(8))), Sets.<ExpressionTree>newIdentityHashSet());
   }
 
   private static void assertNoSharedNodes(ExpressionTree tree,
@@ -310,2596 +320,6 @@ public class TestSearchArgumentImpl {
     }
   }
 
-  @Test
-  public void testExpression1() throws Exception {
-    // first_name = 'john' or
-    //  'greg' < first_name or
-    //  'alan' > first_name or
-    //  id > 12 or
-    //  13 < id or
-    //  id < 15 or
-    //  16 > id or
-    //  (id <=> 30 and first_name <=> 'owen')
-    String exprStr = "<?xml version=\"1.0\" encoding=\"UTF-8\"?> \n" +
-        "<java version=\"1.6.0_31\" class=\"java.beans.XMLDecoder\"> \n" +
-        " <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
-        "  <void property=\"children\"> \n" +
-        "   <object class=\"java.util.ArrayList\"> \n" +
-        "    <void method=\"add\"> \n" +
-        "     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
-        "      <void property=\"children\"> \n" +
-        "       <object class=\"java.util.ArrayList\"> \n" +
-        "        <void method=\"add\"> \n" +
-        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
-        "          <void property=\"children\"> \n" +
-        "           <object class=\"java.util.ArrayList\"> \n" +
-        "            <void method=\"add\"> \n" +
-        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
-        "              <void property=\"children\"> \n" +
-        "               <object class=\"java.util.ArrayList\"> \n" +
-        "                <void method=\"add\"> \n" +
-        "                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
-        "                  <void property=\"children\"> \n" +
-        "                   <object class=\"java.util.ArrayList\"> \n" +
-        "                    <void method=\"add\"> \n" +
-        "                     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
-        "                      <void property=\"children\"> \n" +
-        "                       <object class=\"java.util.ArrayList\"> \n" +
-        "                        <void method=\"add\"> \n" +
-        "                         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
-        "                          <void property=\"children\"> \n" +
-        "                           <object class=\"java.util.ArrayList\"> \n" +
-        "                            <void method=\"add\"> \n" +
-        "                             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
-        "                              <void property=\"children\"> \n" +
-        "                               <object class=\"java.util.ArrayList\"> \n" +
-        "                                <void method=\"add\"> \n" +
-        "                                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
-        "                                  <void property=\"column\"> \n" +
-        "                                   <string>first_name</string> \n" +
-        "                                  </void> \n" +
-        "                                  <void property=\"tabAlias\"> \n" +
-        "                                   <string>orc_people</string> \n" +
-        "                                  </void> \n" +
-        "                                  <void property=\"typeInfo\"> \n" +
-        "                                   <object id=\"PrimitiveTypeInfo0\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
-        "                                    <void property=\"typeName\"> \n" +
-        "                                     <string>string</string> \n" +
-        "                                    </void> \n" +
-        "                                   </object> \n" +
-        "                                  </void> \n" +
-        "                                 </object> \n" +
-        "                                </void> \n" +
-        "                                <void method=\"add\"> \n" +
-        "                                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
-        "                                  <void property=\"typeInfo\"> \n" +
-        "                                   <object idref=\"PrimitiveTypeInfo0\"/> \n" +
-        "                                  </void> \n" +
-        "                                  <void property=\"value\"> \n" +
-        "                                   <string>john</string> \n" +
-        "                                  </void> \n" +
-        "                                 </object> \n" +
-        "                                </void> \n" +
-        "                               </object> \n" +
-        "                              </void> \n" +
-        "                              <void property=\"genericUDF\"> \n" +
-        "                               <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual\"/> \n" +
-        "                              </void> \n" +
-        "                              <void property=\"typeInfo\"> \n" +
-        "                               <object id=\"PrimitiveTypeInfo1\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
-        "                                <void property=\"typeName\"> \n" +
-        "                                 <string>boolean</string> \n" +
-        "                                </void> \n" +
-        "                               </object> \n" +
-        "                              </void> \n" +
-        "                             </object> \n" +
-        "                            </void> \n" +
-        "                            <void method=\"add\"> \n" +
-        "                             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
-        "                              <void property=\"children\"> \n" +
-        "                               <object class=\"java.util.ArrayList\"> \n" +
-        "                                <void method=\"add\"> \n" +
-        "                                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
-        "                                  <void property=\"typeInfo\"> \n" +
-        "                                   <object idref=\"PrimitiveTypeInfo0\"/> \n" +
-        "                                  </void> \n" +
-        "                                  <void property=\"value\"> \n" +
-        "                                   <string>greg</string> \n" +
-        "                                  </void> \n" +
-        "                                 </object> \n" +
-        "                                </void> \n" +
-        "                                <void method=\"add\"> \n" +
-        "                                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
-        "                                  <void property=\"column\"> \n" +
-        "                                   <string>first_name</string> \n" +
-        "                                  </void> \n" +
-        "                                  <void property=\"tabAlias\"> \n" +
-        "                                   <string>orc_people</string> \n" +
-        "                                  </void> \n" +
-        "                                  <void property=\"typeInfo\"> \n" +
-        "                                   <object idref=\"PrimitiveTypeInfo0\"/> \n" +
-        "                                  </void> \n" +
-        "                                 </object> \n" +
-        "                                </void> \n" +
-        "                               </object> \n" +
-        "                              </void> \n" +
-        "                              <void property=\"genericUDF\"> \n" +
-        "                               <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPLessThan\"/> \n" +
-        "                              </void> \n" +
-        "                              <void property=\"typeInfo\"> \n" +
-        "                               <object idref=\"PrimitiveTypeInfo1\"/> \n" +
-        "                              </void> \n" +
-        "                             </object> \n" +
-        "                            </void> \n" +
-        "                           </object> \n" +
-        "                          </void> \n" +
-        "                          <void property=\"genericUDF\"> \n" +
-        "                           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr\"/> \n" +
-        "                          </void> \n" +
-        "                          <void property=\"typeInfo\"> \n" +
-        "                           <object idref=\"PrimitiveTypeInfo1\"/> \n" +
-        "                          </void> \n" +
-        "                         </object> \n" +
-        "                        </void> \n" +
-        "                        <void method=\"add\"> \n" +
-        "                         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
-        "                          <void property=\"children\"> \n" +
-        "                           <object class=\"java.util.ArrayList\"> \n" +
-        "                            <void method=\"add\"> \n" +
-        "                             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
-        "                              <void property=\"typeInfo\"> \n" +
-        "                               <object idref=\"PrimitiveTypeInfo0\"/> \n" +
-        "                              </void> \n" +
-        "                              <void property=\"value\"> \n" +
-        "                               <string>alan</string> \n" +
-        "                              </void> \n" +
-        "                             </object> \n" +
-        "                            </void> \n" +
-        "                            <void method=\"add\"> \n" +
-        "                             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
-        "                              <void property=\"column\"> \n" +
-        "                               <string>first_name</string> \n" +
-        "                              </void> \n" +
-        "                              <void property=\"tabAlias\"> \n" +
-        "                               <string>orc_people</string> \n" +
-        "                              </void> \n" +
-        "                              <void property=\"typeInfo\"> \n" +
-        "                               <object idref=\"PrimitiveTypeInfo0\"/> \n" +
-        "                              </void> \n" +
-        "                             </object> \n" +
-        "                            </void> \n" +
-        "                           </object> \n" +
-        "                          </void> \n" +
-        "                          <void property=\"genericUDF\"> \n" +
-        "                           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPGreaterThan\"/> \n" +
-        "                          </void> \n" +
-        "                          <void property=\"typeInfo\"> \n" +
-        "                           <object idref=\"PrimitiveTypeInfo1\"/> \n" +
-        "                          </void> \n" +
-        "                         </object> \n" +
-        "                        </void> \n" +
-        "                       </object> \n" +
-        "                      </void> \n" +
-        "                      <void property=\"genericUDF\"> \n" +
-        "                       <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr\"/> \n" +
-        "                      </void> \n" +
-        "                      <void property=\"typeInfo\"> \n" +
-        "                       <object idref=\"PrimitiveTypeInfo1\"/> \n" +
-        "                      </void> \n" +
-        "                     </object> \n" +
-        "                    </void> \n" +
-        "                    <void method=\"add\"> \n" +
-        "                     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
-        "                      <void property=\"children\"> \n" +
-        "                       <object class=\"java.util.ArrayList\"> \n" +
-        "                        <void method=\"add\"> \n" +
-        "                         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
-        "                          <void property=\"column\"> \n" +
-        "                           <string>id</string> \n" +
-        "                          </void> \n" +
-        "                          <void property=\"tabAlias\"> \n" +
-        "                           <string>orc_people</string> \n" +
-        "                          </void> \n" +
-        "                          <void property=\"typeInfo\"> \n" +
-        "                           <object id=\"PrimitiveTypeInfo2\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
-        "                            <void property=\"typeName\"> \n" +
-        "                             <string>int</string> \n" +
-        "                            </void> \n" +
-        "                           </object> \n" +
-        "                          </void> \n" +
-        "                         </object> \n" +
-        "                        </void> \n" +
-        "                        <void method=\"add\"> \n" +
-        "                         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
-        "                          <void property=\"typeInfo\"> \n" +
-        "                           <object idref=\"PrimitiveTypeInfo2\"/> \n" +
-        "                          </void> \n" +
-        "                          <void property=\"value\"> \n" +
-        "                           <int>12</int> \n" +
-        "                          </void> \n" +
-        "                         </object> \n" +
-        "                        </void> \n" +
-        "                       </object> \n" +
-        "                      </void> \n" +
-        "                      <void property=\"genericUDF\"> \n" +
-        "                       <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPGreaterThan\"/> \n" +
-        "                      </void> \n" +
-        "                      <void property=\"typeInfo\"> \n" +
-        "                       <object idref=\"PrimitiveTypeInfo1\"/> \n" +
-        "                      </void> \n" +
-        "                     </object> \n" +
-        "                    </void> \n" +
-        "                   </object> \n" +
-        "                  </void> \n" +
-        "                  <void property=\"genericUDF\"> \n" +
-        "                   <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr\"/> \n" +
-        "                  </void> \n" +
-        "                  <void property=\"typeInfo\"> \n" +
-        "                   <object idref=\"PrimitiveTypeInfo1\"/> \n" +
-        "                  </void> \n" +
-        "                 </object> \n" +
-        "                </void> \n" +
-        "                <void method=\"add\"> \n" +
-        "                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
-        "                  <void property=\"children\"> \n" +
-        "                   <object class=\"java.util.ArrayList\"> \n" +
-        "                    <void method=\"add\"> \n" +
-        "                     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
-        "                      <void property=\"typeInfo\"> \n" +
-        "                       <object idref=\"PrimitiveTypeInfo2\"/> \n" +
-        "                      </void> \n" +
-        "                      <void property=\"value\"> \n" +
-        "                       <int>13</int> \n" +
-        "                      </void> \n" +
-        "                     </object> \n" +
-        "                    </void> \n" +
-        "                    <void method=\"add\"> \n" +
-        "                     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
-        "                      <void property=\"column\"> \n" +
-        "                       <string>id</string> \n" +
-        "                      </void> \n" +
-        "                      <void property=\"tabAlias\"> \n" +
-        "                       <string>orc_people</string> \n" +
-        "                      </void> \n" +
-        "                      <void property=\"typeInfo\"> \n" +
-        "                       <object idref=\"PrimitiveTypeInfo2\"/> \n" +
-        "                      </void> \n" +
-        "                     </object> \n" +
-        "                    </void> \n" +
-        "                   </object> \n" +
-        "                  </void> \n" +
-        "                  <void property=\"genericUDF\"> \n" +
-        "                   <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPLessThan\"/> \n" +
-        "                  </void> \n" +
-        "                  <void property=\"typeInfo\"> \n" +
-        "                   <object idref=\"PrimitiveTypeInfo1\"/> \n" +
-        "                  </void> \n" +
-        "                 </object> \n" +
-        "                </void> \n" +
-        "               </object> \n" +
-        "              </void> \n" +
-        "              <void property=\"genericUDF\"> \n" +
-        "               <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr\"/> \n" +
-        "              </void> \n" +
-        "              <void property=\"typeInfo\"> \n" +
-        "               <object idref=\"PrimitiveTypeInfo1\"/> \n" +
-        "              </void> \n" +
-        "             </object> \n" +
-        "            </void> \n" +
-        "            <void method=\"add\"> \n" +
-        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
-        "              <void property=\"children\"> \n" +
-        "               <object class=\"java.util.ArrayList\"> \n" +
-        "                <void method=\"add\"> \n" +
-        "                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
-        "                  <void property=\"column\"> \n" +
-        "                   <string>id</string> \n" +
-        "                  </void> \n" +
-        "                  <void property=\"tabAlias\"> \n" +
-        "                   <string>orc_people</string> \n" +
-        "                  </void> \n" +
-        "                  <void property=\"typeInfo\"> \n" +
-        "                   <object idref=\"PrimitiveTypeInfo2\"/> \n" +
-        "                  </void> \n" +
-        "                 </object> \n" +
-        "                </void> \n" +
-        "                <void method=\"add\"> \n" +
-        "                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
-        "                  <void property=\"typeInfo\"> \n" +
-        "                   <object idref=\"PrimitiveTypeInfo2\"/> \n" +
-        "                  </void> \n" +
-        "                  <void property=\"value\"> \n" +
-        "                   <int>15</int> \n" +
-        "                  </void> \n" +
-        "                 </object> \n" +
-        "                </void> \n" +
-        "               </object> \n" +
-        "              </void> \n" +
-        "              <void property=\"genericUDF\"> \n" +
-        "               <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPLessThan\"/> \n" +
-        "              </void> \n" +
-        "              <void property=\"typeInfo\"> \n" +
-        "               <object idref=\"PrimitiveTypeInfo1\"/> \n" +
-        "              </void> \n" +
-        "             </object> \n" +
-        "            </void> \n" +
-        "           </object> \n" +
-        "          </void> \n" +
-        "          <void property=\"genericUDF\"> \n" +
-        "           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr\"/> \n" +
-        "          </void> \n" +
-        "          <void property=\"typeInfo\"> \n" +
-        "           <object idref=\"PrimitiveTypeInfo1\"/> \n" +
-        "          </void> \n" +
-        "         </object> \n" +
-        "        </void> \n" +
-        "        <void method=\"add\"> \n" +
-        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
-        "          <void property=\"children\"> \n" +
-        "           <object class=\"java.util.ArrayList\"> \n" +
-        "            <void method=\"add\"> \n" +
-        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
-        "              <void property=\"typeInfo\"> \n" +
-        "               <object idref=\"PrimitiveTypeInfo2\"/> \n" +
-        "              </void> \n" +
-        "              <void property=\"value\"> \n" +
-        "               <int>16</int> \n" +
-        "              </void> \n" +
-        "             </object> \n" +
-        "            </void> \n" +
-        "            <void method=\"add\"> \n" +
-        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
-        "              <void property=\"column\"> \n" +
-        "               <string>id</string> \n" +
-        "              </void> \n" +
-        "              <void property=\"tabAlias\"> \n" +
-        "               <string>orc_people</string> \n" +
-        "              </void> \n" +
-        "              <void property=\"typeInfo\"> \n" +
-        "               <object idref=\"PrimitiveTypeInfo2\"/> \n" +
-        "              </void> \n" +
-        "             </object> \n" +
-        "            </void> \n" +
-        "           </object> \n" +
-        "          </void> \n" +
-        "          <void property=\"genericUDF\"> \n" +
-        "           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPGreaterThan\"/> \n" +
-        "          </void> \n" +
-        "          <void property=\"typeInfo\"> \n" +
-        "           <object idref=\"PrimitiveTypeInfo1\"/> \n" +
-        "          </void> \n" +
-        "         </object> \n" +
-        "        </void> \n" +
-        "       </object> \n" +
-        "      </void> \n" +
-        "      <void property=\"genericUDF\"> \n" +
-        "       <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr\"/> \n" +
-        "      </void> \n" +
-        "      <void property=\"typeInfo\"> \n" +
-        "       <object idref=\"PrimitiveTypeInfo1\"/> \n" +
-        "      </void> \n" +
-        "     </object> \n" +
-        "    </void> \n" +
-        "    <void method=\"add\"> \n" +
-        "     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
-        "      <void property=\"children\"> \n" +
-        "       <object class=\"java.util.ArrayList\"> \n" +
-        "        <void method=\"add\"> \n" +
-        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
-        "          <void property=\"children\"> \n" +
-        "           <object class=\"java.util.ArrayList\"> \n" +
-        "            <void method=\"add\"> \n" +
-        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
-        "              <void property=\"column\"> \n" +
-        "               <string>id</string> \n" +
-        "              </void> \n" +
-        "              <void property=\"tabAlias\"> \n" +
-        "               <string>orc_people</string> \n" +
-        "              </void> \n" +
-        "              <void property=\"typeInfo\"> \n" +
-        "               <object idref=\"PrimitiveTypeInfo2\"/> \n" +
-        "              </void> \n" +
-        "             </object> \n" +
-        "            </void> \n" +
-        "            <void method=\"add\"> \n" +
-        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
-        "              <void property=\"typeInfo\"> \n" +
-        "               <object idref=\"PrimitiveTypeInfo2\"/> \n" +
-        "              </void> \n" +
-        "              <void property=\"value\"> \n" +
-        "               <int>30</int> \n" +
-        "              </void> \n" +
-        "             </object> \n" +
-        "            </void> \n" +
-        "           </object> \n" +
-        "          </void> \n" +
-        "          <void property=\"genericUDF\"> \n" +
-        "           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualNS\"/> \n" +
-        "          </void> \n" +
-        "          <void property=\"typeInfo\"> \n" +
-        "           <object idref=\"PrimitiveTypeInfo1\"/> \n" +
-        "          </void> \n" +
-        "         </object> \n" +
-        "        </void> \n" +
-        "        <void method=\"add\"> \n" +
-        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
-        "          <void property=\"children\"> \n" +
-        "           <object class=\"java.util.ArrayList\"> \n" +
-        "            <void method=\"add\"> \n" +
-        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
-        "              <void property=\"column\"> \n" +
-        "               <string>first_name</string> \n" +
-        "              </void> \n" +
-        "              <void property=\"tabAlias\"> \n" +
-        "               <string>orc_people</string> \n" +
-        "              </void> \n" +
-        "              <void property=\"typeInfo\"> \n" +
-        "               <object idref=\"PrimitiveTypeInfo0\"/> \n" +
-        "              </void> \n" +
-        "             </object> \n" +
-        "            </void> \n" +
-        "            <void method=\"add\"> \n" +
-        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
-        "              <void property=\"typeInfo\"> \n" +
-        "               <object idref=\"PrimitiveTypeInfo0\"/> \n" +
-        "              </void> \n" +
-        "              <void property=\"value\"> \n" +
-        "               <string>owen</string> \n" +
-        "              </void> \n" +
-        "             </object> \n" +
-        "            </void> \n" +
-        "           </object> \n" +
-        "          </void> \n" +
-        "          <void property=\"genericUDF\"> \n" +
-        "           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualNS\"/> \n" +
-        "          </void> \n" +
-        "          <void property=\"typeInfo\"> \n" +
-        "           <object idref=\"PrimitiveTypeInfo1\"/> \n" +
-        "          </void> \n" +
-        "         </object> \n" +
-        "        </void> \n" +
-        "       </object> \n" +
-        "      </void> \n" +
-        "      <void property=\"genericUDF\"> \n" +
-        "       <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPAnd\"/> \n" +
-        "      </void> \n" +
-        "      <void property=\"typeInfo\"> \n" +
-        "       <object idref=\"PrimitiveTypeInfo1\"/> \n" +
-        "      </void> \n" +
-        "     </object> \n" +
-        "    </void> \n" +
-        "   </object> \n" +
-        "  </void> \n" +
-        "  <void property=\"genericUDF\"> \n" +
-        "   <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr\"/> \n" +
-        "  </void> \n" +
-        "  <void property=\"typeInfo\"> \n" +
-        "   <object idref=\"PrimitiveTypeInfo1\"/> \n" +
-        "  </void> \n" +
-        " </object> \n" +
-        "</java> \n";
-
-    SearchArgumentImpl sarg =
-        (SearchArgumentImpl) SearchArgumentFactory.create(getFuncDesc(exprStr));
-    List<PredicateLeaf> leaves = sarg.getLeaves();
-    assertEquals(9, leaves.size());
-
-    FilterPredicate p = ParquetRecordReaderWrapper.toFilterPredicate(sarg);
-    String[] conditions = new String[]{
-      "eq(first_name, Binary{\"john\"})",    /* first_name = 'john' */
-      "not(lteq(first_name, Binary{\"greg\"}))", /* 'greg' < first_name */
-      "lt(first_name, Binary{\"alan\"})",   /* 'alan' > first_name */
-      "not(lteq(id, 12))",                  /* id > 12 or */
-      "not(lteq(id, 13))",                  /* 13 < id or */
-      "lt(id, 15)",                         /* id < 15 or */
-      "lt(id, 16)",                         /* 16 > id or */
-      "eq(id, 30)",                         /* id <=> 30 */
-      "eq(first_name, Binary{\"owen\"})"    /* first_name <=> 'owen' */
-    };
-    String expected = String
-      .format("and(or(or(or(or(or(or(or(%1$s, %2$s), %3$s), %4$s), %5$s), %6$s), %7$s), %8$s), " +
-        "or(or(or(or(or(or(or(%1$s, %2$s), %3$s), %4$s), %5$s), %6$s), %7$s), %9$s))", conditions);
-    assertEquals(expected, p.toString());
-
-    PredicateLeaf leaf = leaves.get(0);
-    assertEquals(PredicateLeaf.Type.STRING, leaf.getType());
-    assertEquals(PredicateLeaf.Operator.EQUALS, leaf.getOperator());
-    assertEquals("first_name", leaf.getColumnName());
-    assertEquals("john", leaf.getLiteral());
-
-    leaf = leaves.get(1);
-    assertEquals(PredicateLeaf.Type.STRING, leaf.getType());
-    assertEquals(PredicateLeaf.Operator.LESS_THAN_EQUALS, leaf.getOperator());
-    assertEquals("first_name", leaf.getColumnName());
-    assertEquals("greg", leaf.getLiteral());
-
-    leaf = leaves.get(2);
-    assertEquals(PredicateLeaf.Type.STRING, leaf.getType());
-    assertEquals(PredicateLeaf.Operator.LESS_THAN, leaf.getOperator());
-    assertEquals("first_name", leaf.getColumnName());
-    assertEquals("alan", leaf.getLiteral());
-
-    leaf = leaves.get(3);
-    assertEquals(PredicateLeaf.Type.INTEGER, leaf.getType());
-    assertEquals(PredicateLeaf.Operator.LESS_THAN_EQUALS, leaf.getOperator());
-    assertEquals("id", leaf.getColumnName());
-    assertEquals(12, leaf.getLiteral());
-
-    leaf = leaves.get(4);
-    assertEquals(PredicateLeaf.Type.INTEGER, leaf.getType());
-    assertEquals(PredicateLeaf.Operator.LESS_THAN_EQUALS, leaf.getOperator());
-    assertEquals("id", leaf.getColumnName());
-    assertEquals(13, leaf.getLiteral());
-
-    leaf = leaves.get(5);
-    assertEquals(PredicateLeaf.Type.INTEGER, leaf.getType());
-    assertEquals(PredicateLeaf.Operator.LESS_THAN, leaf.getOperator());
-    assertEquals("id", leaf.getColumnName());
-    assertEquals(15, leaf.getLiteral());
-
-    leaf = leaves.get(6);
-    assertEquals(PredicateLeaf.Type.INTEGER, leaf.getType());
-    assertEquals(PredicateLeaf.Operator.LESS_THAN, leaf.getOperator());
-    assertEquals("id", leaf.getColumnName());
-    assertEquals(16, leaf.getLiteral());
-
-    leaf = leaves.get(7);
-    assertEquals(PredicateLeaf.Type.INTEGER, leaf.getType());
-    assertEquals(PredicateLeaf.Operator.NULL_SAFE_EQUALS, leaf.getOperator());
-    assertEquals("id", leaf.getColumnName());
-    assertEquals(30, leaf.getLiteral());
-
-    leaf = leaves.get(8);
-    assertEquals(PredicateLeaf.Type.STRING, leaf.getType());
-    assertEquals(PredicateLeaf.Operator.NULL_SAFE_EQUALS, leaf.getOperator());
-    assertEquals("first_name", leaf.getColumnName());
-    assertEquals("owen", leaf.getLiteral());
-
-    assertEquals("(and (or leaf-0 (not leaf-1) leaf-2 (not leaf-3)" +
-        " (not leaf-4) leaf-5 leaf-6 leaf-7)" +
-        " (or leaf-0 (not leaf-1) leaf-2 (not leaf-3)" +
-        " (not leaf-4) leaf-5 leaf-6 leaf-8))",
-        sarg.getExpression().toString());
-    assertNoSharedNodes(sarg.getExpression(),
-        Sets.<ExpressionTree>newIdentityHashSet());
-  }
-
-  @Test
-  public void testExpression2() throws Exception {
-    /* first_name is null or
-       first_name <> 'sue' or
-       id >= 12 or
-       id <= 4; */
-    String exprStr = "<?xml version=\"1.0\" encoding=\"UTF-8\"?> \n" +
-        "<java version=\"1.6.0_31\" class=\"java.beans.XMLDecoder\"> \n" +
-        " <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
-        "  <void property=\"children\"> \n" +
-        "   <object class=\"java.util.ArrayList\"> \n" +
-        "    <void method=\"add\"> \n" +
-        "     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
-        "      <void property=\"children\"> \n" +
-        "       <object class=\"java.util.ArrayList\"> \n" +
-        "        <void method=\"add\"> \n" +
-        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
-        "          <void property=\"children\"> \n" +
-        "           <object class=\"java.util.ArrayList\"> \n" +
-        "            <void method=\"add\"> \n" +
-        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
-        "              <void property=\"children\"> \n" +
-        "               <object class=\"java.util.ArrayList\"> \n" +
-        "                <void method=\"add\"> \n" +
-        "                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
-        "                  <void property=\"column\"> \n" +
-        "                   <string>first_name</string> \n" +
-        "                  </void> \n" +
-        "                  <void property=\"tabAlias\"> \n" +
-        "                   <string>orc_people</string> \n" +
-        "                  </void> \n" +
-        "                  <void property=\"typeInfo\"> \n" +
-        "                   <object id=\"PrimitiveTypeInfo0\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
-        "                    <void property=\"typeName\"> \n" +
-        "                     <string>string</string> \n" +
-        "                    </void> \n" +
-        "                   </object> \n" +
-        "                  </void> \n" +
-        "                 </object> \n" +
-        "                </void> \n" +
-        "               </object> \n" +
-        "              </void> \n" +
-        "              <void property=\"genericUDF\"> \n" +
-        "               <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNull\"/> \n" +
-        "              </void> \n" +
-        "              <void property=\"typeInfo\"> \n" +
-        "               <object id=\"PrimitiveTypeInfo1\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
-        "                <void property=\"typeName\"> \n" +
-        "                 <string>boolean</string> \n" +
-        "                </void> \n" +
-        "               </object> \n" +
-        "              </void> \n" +
-        "             </object> \n" +
-        "            </void> \n" +
-        "            <void method=\"add\"> \n" +
-        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
-        "              <void property=\"children\"> \n" +
-        "               <object class=\"java.util.ArrayList\"> \n" +
-        "                <void method=\"add\"> \n" +
-        "                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
-        "                  <void property=\"column\"> \n" +
-        "                   <string>first_name</string> \n" +
-        "                  </void> \n" +
-        "                  <void property=\"tabAlias\"> \n" +
-        "                   <string>orc_people</string> \n" +
-        "                  </void> \n" +
-        "                  <void property=\"typeInfo\"> \n" +
-        "                   <object idref=\"PrimitiveTypeInfo0\"/> \n" +
-        "                  </void> \n" +
-        "                 </object> \n" +
-        "                </void> \n" +
-        "                <void method=\"add\"> \n" +
-        "                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
-        "                  <void property=\"typeInfo\"> \n" +
-        "                   <object idref=\"PrimitiveTypeInfo0\"/> \n" +
-        "                  </void> \n" +
-        "                  <void property=\"value\"> \n" +
-        "                   <string>sue</string> \n" +
-        "                  </void> \n" +
-        "                 </object> \n" +
-        "                </void> \n" +
-        "               </object> \n" +
-        "              </void> \n" +
-        "              <void property=\"genericUDF\"> \n" +
-        "               <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNotEqual\"/> \n" +
-        "              </void> \n" +
-        "              <void property=\"typeInfo\"> \n" +
-        "               <object idref=\"PrimitiveTypeInfo1\"/> \n" +
-        "              </void> \n" +
-        "             </object> \n" +
-        "            </void> \n" +
-        "           </object> \n" +
-        "          </void> \n" +
-        "          <void property=\"genericUDF\"> \n" +
-        "           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr\"/> \n" +
-        "          </void> \n" +
-        "          <void property=\"typeInfo\"> \n" +
-        "           <object idref=\"PrimitiveTypeInfo1\"/> \n" +
-        "          </void> \n" +
-        "         </object> \n" +
-        "        </void> \n" +
-        "        <void method=\"add\"> \n" +
-        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
-        "          <void property=\"children\"> \n" +
-        "           <object class=\"java.util.ArrayList\"> \n" +
-        "            <void method=\"add\"> \n" +
-        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
-        "              <void property=\"column\"> \n" +
-        "               <string>id</string> \n" +
-        "              </void> \n" +
-        "              <void property=\"tabAlias\"> \n" +
-        "               <string>orc_people</string> \n" +
-        "              </void> \n" +
-        "              <void property=\"typeInfo\"> \n" +
-        "               <object id=\"PrimitiveTypeInfo2\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
-        "                <void property=\"typeName\"> \n" +
-        "                 <string>int</string> \n" +
-        "                </void> \n" +
-        "               </object> \n" +
-        "              </void> \n" +
-        "             </object> \n" +
-        "            </void> \n" +
-        "            <void method=\"add\"> \n" +
-        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
-        "              <void property=\"typeInfo\"> \n" +
-        "               <object idref=\"PrimitiveTypeInfo2\"/> \n" +
-        "              </void> \n" +
-        "              <void property=\"value\"> \n" +
-        "               <int>12</int> \n" +
-        "              </void> \n" +
-        "             </object> \n" +
-        "            </void> \n" +
-        "           </object> \n" +
-        "          </void> \n" +
-        "          <void property=\"genericUDF\"> \n" +
-        "           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrGreaterThan\"/> \n" +
-        "          </void> \n" +
-        "          <void property=\"typeInfo\"> \n" +
-        "           <object idref=\"PrimitiveTypeInfo1\"/> \n" +
-        "          </void> \n" +
-        "         </object> \n" +
-        "        </void> \n" +
-        "       </object> \n" +
-        "      </void> \n" +
-        "      <void property=\"genericUDF\"> \n" +
-        "       <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr\"/> \n" +
-        "      </void> \n" +
-        "      <void property=\"typeInfo\"> \n" +
-        "       <object idref=\"PrimitiveTypeInfo1\"/> \n" +
-        "      </void> \n" +
-        "     </object> \n" +
-        "    </void> \n" +
-        "    <void method=\"add\"> \n" +
-        "     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
-        "      <void property=\"children\"> \n" +
-        "       <object class=\"java.util.ArrayList\"> \n" +
-        "        <void method=\"add\"> \n" +
-        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
-        "          <void property=\"column\"> \n" +
-        "           <string>id</string> \n" +
-        "          </void> \n" +
-        "          <void property=\"tabAlias\"> \n" +
-        "           <string>orc_people</string> \n" +
-        "          </void> \n" +
-        "          <void property=\"typeInfo\"> \n" +
-        "           <object idref=\"PrimitiveTypeInfo2\"/> \n" +
-        "          </void> \n" +
-        "         </object> \n" +
-        "        </void> \n" +
-        "        <void method=\"add\"> \n" +
-        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
-        "          <void property=\"typeInfo\"> \n" +
-        "           <object idref=\"PrimitiveTypeInfo2\"/> \n" +
-        "          </void> \n" +
-        "          <void property=\"value\"> \n" +
-        "           <int>4</int> \n" +
-        "          </void> \n" +
-        "         </object> \n" +
-        "        </void> \n" +
-        "       </object> \n" +
-        "      </void> \n" +
-        "      <void property=\"genericUDF\"> \n" +
-        "       <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrLessThan\"/> \n" +
-        "      </void> \n" +
-        "      <void property=\"typeInfo\"> \n" +
-        "       <object idref=\"PrimitiveTypeInfo1\"/> \n" +
-        "      </void> \n" +
-        "     </object> \n" +
-        "    </void> \n" +
-        "   </object> \n" +
-        "  </void> \n" +
-        "  <void property=\"genericUDF\"> \n" +
-        "   <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr\"/> \n" +
-        "  </void> \n" +
-        "  <void property=\"typeInfo\"> \n" +
-        "   <object idref=\"PrimitiveTypeInfo1\"/> \n" +
-        "  </void> \n" +
-        " </object> \n" +
-        "</java> \n";
-
-    SearchArgumentImpl sarg =
-        (SearchArgumentImpl) SearchArgumentFactory.create(getFuncDesc(exprStr));
-    List<PredicateLeaf> leaves = sarg.getLeaves();
-    assertEquals(4, leaves.size());
-
-    String[] conditions = new String[]{
-      "eq(first_name, null)",               /* first_name is null  */
-      "not(eq(first_name, Binary{\"sue\"}))",    /* first_name <> 'sue' */
-      "not(lt(id, 12))",                    /* id >= 12            */
-      "lteq(id, 4)"                         /* id <= 4             */
-    };
-
-    FilterPredicate p = ParquetRecordReaderWrapper.toFilterPredicate(sarg);
-    String expected = String.format("or(or(or(%1$s, %2$s), %3$s), %4$s)", conditions);
-    assertEquals(expected, p.toString());
-
-    PredicateLeaf leaf = leaves.get(0);
-    assertEquals(PredicateLeaf.Type.STRING, leaf.getType());
-    assertEquals(PredicateLeaf.Operator.IS_NULL, leaf.getOperator());
-    assertEquals("first_name", leaf.getColumnName());
-    assertEquals(null, leaf.getLiteral());
-    assertEquals(null, leaf.getLiteralList());
-
-    leaf = leaves.get(1);
-    assertEquals(PredicateLeaf.Type.STRING, leaf.getType());
-    assertEquals(PredicateLeaf.Operator.EQUALS, leaf.getOperator());
-    assertEquals("first_name", leaf.getColumnName());
-    assertEquals("sue", leaf.getLiteral());
-
-    leaf = leaves.get(2);
-    assertEquals(PredicateLeaf.Type.INTEGER, leaf.getType());
-    assertEquals(PredicateLeaf.Operator.LESS_THAN, leaf.getOperator());
-    assertEquals("id", leaf.getColumnName());
-    assertEquals(12, leaf.getLiteral());
-
-    leaf = leaves.get(3);
-    assertEquals(PredicateLeaf.Type.INTEGER, leaf.getType());
-    assertEquals(PredicateLeaf.Operator.LESS_THAN_EQUALS, leaf.getOperator());
-    assertEquals("id", leaf.getColumnName());
-    assertEquals(4, leaf.getLiteral());
-
-    assertEquals("(or leaf-0 (not leaf-1) (not leaf-2) leaf-3)",
-        sarg.getExpression().toString());
-    assertNoSharedNodes(sarg.getExpression(),
-        Sets.<ExpressionTree>newIdentityHashSet());
-    assertEquals(TruthValue.NO,
-        sarg.evaluate(values(TruthValue.NO, TruthValue.YES, TruthValue.YES,
-            TruthValue.NO)));
-    assertEquals(TruthValue.YES,
-        sarg.evaluate(values(TruthValue.YES, TruthValue.YES, TruthValue.YES,
-            TruthValue.NO)));
-    assertEquals(TruthValue.YES,
-        sarg.evaluate(values(TruthValue.NO, TruthValue.NO, TruthValue.YES,
-            TruthValue.NO)));
-    assertEquals(TruthValue.YES,
-        sarg.evaluate(values(TruthValue.NO, TruthValue.YES, TruthValue.NO,
-            TruthValue.NO)));
-    assertEquals(TruthValue.YES,
-        sarg.evaluate(values(TruthValue.NO, TruthValue.YES, TruthValue.YES,
-            TruthValue.YES)));
-    assertEquals(TruthValue.NULL,
-        sarg.evaluate(values(TruthValue.NULL, TruthValue.YES, TruthValue.YES,
-            TruthValue.NO)));
-    assertEquals(TruthValue.NULL,
-        sarg.evaluate(values(TruthValue.NO, TruthValue.NULL, TruthValue.YES,
-            TruthValue.NO)));
-    assertEquals(TruthValue.NULL,
-        sarg.evaluate(values(TruthValue.NO, TruthValue.YES, TruthValue.NULL,
-            TruthValue.NO)));
-    assertEquals(TruthValue.NULL,
-        sarg.evaluate(values(TruthValue.NO, TruthValue.YES, TruthValue.YES,
-            TruthValue.NULL)));
-    assertEquals(TruthValue.YES_NO,
-        sarg.evaluate(values(TruthValue.NO, TruthValue.YES_NO, TruthValue.YES,
-            TruthValue.YES_NO)));
-    assertEquals(TruthValue.NO_NULL,
-        sarg.evaluate(values(TruthValue.NO, TruthValue.YES_NULL, TruthValue.YES,
-            TruthValue.NO_NULL)));
-    assertEquals(TruthValue.YES_NULL,
-        sarg.evaluate(values(TruthValue.YES_NULL, TruthValue.YES_NO_NULL,
-            TruthValue.YES, TruthValue.NULL)));
-    assertEquals(TruthValue.YES_NO_NULL,
-        sarg.evaluate(values(TruthValue.NO_NULL, TruthValue.YES_NO_NULL,
-            TruthValue.YES, TruthValue.NO)));
-  }
-
-  @Test
-  public void testExpression3() throws Exception {
-    /* (id between 23 and 45) and
-       first_name = 'alan' and
-       substr('xxxxx', 3) == first_name and
-       'smith' = last_name and
-       substr(first_name, 3) == 'yyy' */
-    String exprStr = "<?xml version=\"1.0\" encoding=\"UTF-8\"?> \n" +
-        "<java version=\"1.6.0_31\" class=\"java.beans.XMLDecoder\"> \n" +
-        " <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
-        "  <void property=\"children\"> \n" +
-        "   <object class=\"java.util.ArrayList\"> \n" +
-        "    <void method=\"add\"> \n" +
-        "     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
-        "      <void property=\"children\"> \n" +
-        "       <object class=\"java.util.ArrayList\"> \n" +
-        "        <void method=\"add\"> \n" +
-        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
-        "          <void property=\"children\"> \n" +
-        "           <object class=\"java.util.ArrayList\"> \n" +
-        "            <void method=\"add\"> \n" +
-        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
-        "              <void property=\"children\"> \n" +
-        "               <object class=\"java.util.ArrayList\"> \n" +
-        "                <void method=\"add\"> \n" +
-        "                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
-        "                  <void property=\"children\"> \n" +
-        "                   <object class=\"java.util.ArrayList\"> \n" +
-        "                    <void method=\"add\"> \n" +
-        "                     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
-        "                      <void property=\"typeInfo\"> \n" +
-        "                       <object id=\"PrimitiveTypeInfo0\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
-        "                        <void property=\"typeName\"> \n" +
-        "                         <string>boolean</string> \n" +
-        "                        </void> \n" +
-        "                       </object> \n" +
-        "                      </void> \n" +
-        "                      <void property=\"value\"> \n" +
-        "                       <boolean>false</boolean> \n" +
-        "                      </void> \n" +
-        "                     </object> \n" +
-        "                    </void> \n" +
-        "                    <void method=\"add\"> \n" +
-        "                     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
-        "                      <void property=\"column\"> \n" +
-        "                       <string>id</string> \n" +
-        "                      </void> \n" +
-        "                      <void property=\"tabAlias\"> \n" +
-        "                       <string>orc_people</string> \n" +
-        "                      </void> \n" +
-        "                      <void property=\"typeInfo\"> \n" +
-        "                       <object id=\"PrimitiveTypeInfo1\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
-        "                        <void property=\"typeName\"> \n" +
-        "                         <string>int</string> \n" +
-        "                        </void> \n" +
-        "                       </object> \n" +
-        "                      </void> \n" +
-        "                     </object> \n" +
-        "                    </void> \n" +
-        "                    <void method=\"add\"> \n" +
-        "                     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
-        "                      <void property=\"typeInfo\"> \n" +
-        "                       <object idref=\"PrimitiveTypeInfo1\"/> \n" +
-        "                      </void> \n" +
-        "                      <void property=\"value\"> \n" +
-        "                       <int>23</int> \n" +
-        "                      </void> \n" +
-        "                     </object> \n" +
-        "                    </void> \n" +
-        "                    <void method=\"add\"> \n" +
-        "                     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
-        "                      <void property=\"typeInfo\"> \n" +
-        "                       <object idref=\"PrimitiveTypeInfo1\"/> \n" +
-        "                      </void> \n" +
-        "                      <void property=\"value\"> \n" +
-        "                       <int>45</int> \n" +
-        "                      </void> \n" +
-        "                     </object> \n" +
-        "                    </void> \n" +
-        "                   </object> \n" +
-        "                  </void> \n" +
-        "                  <void property=\"genericUDF\"> \n" +
-        "                   <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFBetween\"/> \n" +
-        "                  </void> \n" +
-        "                  <void property=\"typeInfo\"> \n" +
-        "                   <object idref=\"PrimitiveTypeInfo0\"/> \n" +
-        "                  </void> \n" +
-        "                 </object> \n" +
-        "                </void> \n" +
-        "                <void method=\"add\"> \n" +
-        "                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
-        "                  <void property=\"children\"> \n" +
-        "                   <object class=\"java.util.ArrayList\"> \n" +
-        "                    <void method=\"add\"> \n" +
-        "                     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
-        "                      <void property=\"column\"> \n" +
-        "                       <string>first_name</string> \n" +
-        "                      </void> \n" +
-        "                      <void property=\"tabAlias\"> \n" +
-        "                       <string>orc_people</string> \n" +
-        "                      </void> \n" +
-        "                      <void property=\"typeInfo\"> \n" +
-        "                       <object id=\"PrimitiveTypeInfo2\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
-        "                        <void property=\"typeName\"> \n" +
-        "                         <string>string</string> \n" +
-        "                        </void> \n" +
-        "                       </object> \n" +
-        "                      </void> \n" +
-        "                     </object> \n" +
-        "                    </void> \n" +
-        "                    <void method=\"add\"> \n" +
-        "                     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
-        "                      <void property=\"typeInfo\"> \n" +
-        "                       <object idref=\"PrimitiveTypeInfo2\"/> \n" +
-        "                      </void> \n" +
-        "                      <void property=\"value\"> \n" +
-        "                       <string>alan</string> \n" +
-        "                      </void> \n" +
-        "                     </object> \n" +
-        "                    </void> \n" +
-        "                   </object> \n" +
-        "                  </void> \n" +
-        "                  <void property=\"genericUDF\"> \n" +
-        "                   <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual\"/> \n" +
-        "                  </void> \n" +
-        "                  <void property=\"typeInfo\"> \n" +
-        "                   <object idref=\"PrimitiveTypeInfo0\"/> \n" +
-        "                  </void> \n" +
-        "                 </object> \n" +
-        "                </void> \n" +
-        "               </object> \n" +
-        "              </void> \n" +
-        "              <void property=\"genericUDF\"> \n" +
-        "               <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPAnd\"/> \n" +
-        "              </void> \n" +
-        "              <void property=\"typeInfo\"> \n" +
-        "               <object idref=\"PrimitiveTypeInfo0\"/> \n" +
-        "              </void> \n" +
-        "             </object> \n" +
-        "            </void> \n" +
-        "            <void method=\"add\"> \n" +
-        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
-        "              <void property=\"children\"> \n" +
-        "               <object class=\"java.util.ArrayList\"> \n" +
-        "                <void method=\"add\"> \n" +
-        "                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
-        "                  <void property=\"children\"> \n" +
-        "                   <object class=\"java.util.ArrayList\"> \n" +
-        "                    <void method=\"add\"> \n" +
-        "                     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
-        "                      <void property=\"typeInfo\"> \n" +
-        "                       <object idref=\"PrimitiveTypeInfo2\"/> \n" +
-        "                      </void> \n" +
-        "                      <void property=\"value\"> \n" +
-        "                       <string>xxxxx</string> \n" +
-        "                      </void> \n" +
-        "                     </object> \n" +
-        "                    </void> \n" +
-        "                    <void method=\"add\"> \n" +
-        "                     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
-        "                      <void property=\"typeInfo\"> \n" +
-        "                       <object idref=\"PrimitiveTypeInfo1\"/> \n" +
-        "                      </void> \n" +
-        "                      <void property=\"value\"> \n" +
-        "                       <int>3</int> \n" +
-        "                      </void> \n" +
-        "                     </object> \n" +
-        "                    </void> \n" +
-        "                   </object> \n" +
-        "                  </void> \n" +
-        "                  <void property=\"genericUDF\"> \n" +
-        "                   <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge\"> \n" +
-        "                    <void property=\"udfClassName\"> \n" +
-        "                     <string>org.apache.hadoop.hive.ql.udf.UDFSubstr</string> \n" +
-        "                    </void> \n" +
-        "                    <void property=\"udfName\"> \n" +
-        "                     <string>substr</string> \n" +
-        "                    </void> \n" +
-        "                   </object> \n" +
-        "                  </void> \n" +
-        "                  <void property=\"typeInfo\"> \n" +
-        "                   <object idref=\"PrimitiveTypeInfo2\"/> \n" +
-        "                  </void> \n" +
-        "                 </object> \n" +
-        "                </void> \n" +
-        "                <void method=\"add\"> \n" +
-        "                 <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
-        "                  <void property=\"column\"> \n" +
-        "                   <string>first_name</string> \n" +
-        "                  </void> \n" +
-        "                  <void property=\"tabAlias\"> \n" +
-        "                   <string>orc_people</string> \n" +
-        "                  </void> \n" +
-        "                  <void property=\"typeInfo\"> \n" +
-        "                   <object idref=\"PrimitiveTypeInfo2\"/> \n" +
-        "                  </void> \n" +
-        "                 </object> \n" +
-        "                </void> \n" +
-        "               </object> \n" +
-        "              </void> \n" +
-        "              <void property=\"genericUDF\"> \n" +
-        "               <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual\"/> \n" +
-        "              </void> \n" +
-        "              <void property=\"typeInfo\"> \n" +
-        "               <object idref=\"PrimitiveTypeInfo0\"/> \n" +
-        "              </void> \n" +
-        "             </object> \n" +
-        "            </void> \n" +
-        "           </object> \n" +
-        "          </void> \n" +
-        "          <void property=\"genericUDF\"> \n" +
-        "           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPAnd\"/> \n" +
-        "          </void> \n" +
-        "          <void property=\"typeInfo\"> \n" +
-        "           <object idref=\"PrimitiveTypeInfo0\"/> \n" +
-        "          </void> \n" +
-        "         </object> \n" +
-        "        </void> \n" +
-        "        <void method=\"add\"> \n" +
-        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
-        "          <void property=\"children\"> \n" +
-        "           <object class=\"java.util.ArrayList\"> \n" +
-        "            <void method=\"add\"> \n" +
-        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
-        "              <void property=\"typeInfo\"> \n" +
-        "               <object idref=\"PrimitiveTypeInfo2\"/> \n" +
-        "              </void> \n" +
-        "              <void property=\"value\"> \n" +
-        "               <string>smith</string> \n" +
-        "              </void> \n" +
-        "             </object> \n" +
-        "            </void> \n" +
-        "            <void method=\"add\"> \n" +
-        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
-        "              <void property=\"column\"> \n" +
-        "               <string>last_name</string> \n" +
-        "              </void> \n" +
-        "              <void property=\"tabAlias\"> \n" +
-        "               <string>orc_people</string> \n" +
-        "              </void> \n" +
-        "              <void property=\"typeInfo\"> \n" +
-        "               <object idref=\"PrimitiveTypeInfo2\"/> \n" +
-        "              </void> \n" +
-        "             </object> \n" +
-        "            </void> \n" +
-        "           </object> \n" +
-        "          </void> \n" +
-        "          <void property=\"genericUDF\"> \n" +
-        "           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual\"/> \n" +
-        "          </void> \n" +
-        "          <void property=\"typeInfo\"> \n" +
-        "           <object idref=\"PrimitiveTypeInfo0\"/> \n" +
-        "          </void> \n" +
-        "         </object> \n" +
-        "        </void> \n" +
-        "       </object> \n" +
-        "      </void> \n" +
-        "      <void property=\"genericUDF\"> \n" +
-        "       <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPAnd\"/> \n" +
-        "      </void> \n" +
-        "      <void property=\"typeInfo\"> \n" +
-        "       <object idref=\"PrimitiveTypeInfo0\"/> \n" +
-        "      </void> \n" +
-        "     </object> \n" +
-        "    </void> \n" +
-        "    <void method=\"add\"> \n" +
-        "     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
-        "      <void property=\"children\"> \n" +
-        "       <object class=\"java.util.ArrayList\"> \n" +
-        "        <void method=\"add\"> \n" +
-        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
-        "          <void property=\"children\"> \n" +
-        "           <object class=\"java.util.ArrayList\"> \n" +
-        "            <void method=\"add\"> \n" +
-        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
-        "              <void property=\"column\"> \n" +
-        "               <string>first_name</string> \n" +
-        "              </void> \n" +
-        "              <void property=\"tabAlias\"> \n" +
-        "               <string>orc_people</string> \n" +
-        "              </void> \n" +
-        "              <void property=\"typeInfo\"> \n" +
-        "               <object idref=\"PrimitiveTypeInfo2\"/> \n" +
-        "              </void> \n" +
-        "             </object> \n" +
-        "            </void> \n" +
-        "            <void method=\"add\"> \n" +
-        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
-        "              <void property=\"typeInfo\"> \n" +
-        "               <object idref=\"PrimitiveTypeInfo1\"/> \n" +
-        "              </void> \n" +
-        "              <void property=\"value\"> \n" +
-        "               <int>3</int> \n" +
-        "              </void> \n" +
-        "             </object> \n" +
-        "            </void> \n" +
-        "           </object> \n" +
-        "          </void> \n" +
-        "          <void property=\"genericUDF\"> \n" +
-        "           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge\"> \n" +
-        "            <void property=\"udfClassName\"> \n" +
-        "             <string>org.apache.hadoop.hive.ql.udf.UDFSubstr</string> \n" +
-        "            </void> \n" +
-        "            <void property=\"udfName\"> \n" +
-        "             <string>substr</string> \n" +
-        "            </void> \n" +
-        "           </object> \n" +
-        "          </void> \n" +
-        "          <void property=\"typeInfo\"> \n" +
-        "           <object idref=\"PrimitiveTypeInfo2\"/> \n" +
-        "          </void> \n" +
-        "         </object> \n" +
-        "        </void> \n" +
-        "        <void method=\"add\"> \n" +
-        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
-        "          <void property=\"typeInfo\"> \n" +
-        "           <object idref=\"PrimitiveTypeInfo2\"/> \n" +
-        "          </void> \n" +
-        "          <void property=\"value\"> \n" +
-        "           <string>yyy</string> \n" +
-        "          </void> \n" +
-        "         </object> \n" +
-        "        </void> \n" +
-        "       </object> \n" +
-        "      </void> \n" +
-        "      <void property=\"genericUDF\"> \n" +
-        "       <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual\"/> \n" +
-        "      </void> \n" +
-        "      <void property=\"typeInfo\"> \n" +
-        "       <object idref=\"PrimitiveTypeInfo0\"/> \n" +
-        "      </void> \n" +
-        "     </object> \n" +
-        "    </void> \n" +
-        "   </object> \n" +
-        "  </void> \n" +
-        "  <void property=\"genericUDF\"> \n" +
-        "   <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPAnd\"/> \n" +
-        "  </void> \n" +
-        "  <void property=\"typeInfo\"> \n" +
-        "   <object idref=\"PrimitiveTypeInfo0\"/> \n" +
-        "  </void> \n" +
-        " </object> \n" +
-        "</java> \n";
-
-    SearchArgumentImpl sarg =
-        (SearchArgumentImpl) SearchArgumentFactory.create(getFuncDesc(exprStr));
-    List<PredicateLeaf> leaves = sarg.getLeaves();
-    assertEquals(3, leaves.size());
-
-    String[] conditions = new String[]{
-      "lt(id, 45)",                         /* id between 23 and 45 */
-      "not(lteq(id, 23))",                   /* id between 23 and 45 */
-      "eq(first_name, Binary{\"alan\"})",   /* first_name = 'alan'  */
-      "eq(last_name, Binary{\"smith\"})"    /* 'smith' = last_name  */
-    };
-
-    FilterPredicate p = ParquetRecordReaderWrapper.toFilterPredicate(sarg);
-    String expected = String.format("and(and(and(%1$s, %2$s), %3$s), %4$s)", conditions);
-    assertEquals(expected, p.toString());
-
-    PredicateLeaf leaf = leaves.get(0);
-    assertEquals(PredicateLeaf.Type.INTEGER, leaf.getType());
-    assertEquals(PredicateLeaf.Operator.BETWEEN, leaf.getOperator());
-    assertEquals("id", leaf.getColumnName());
-    assertEquals(null, leaf.getLiteral());
-    assertEquals(23, leaf.getLiteralList().get(0));
-    assertEquals(45, leaf.getLiteralList().get(1));
-
-    leaf = leaves.get(1);
-    assertEquals(PredicateLeaf.Type.STRING, leaf.getType());
-    assertEquals(PredicateLeaf.Operator.EQUALS, leaf.getOperator());
-    assertEquals("first_name", leaf.getColumnName());
-    assertEquals("alan", leaf.getLiteral());
-
-    leaf = leaves.get(2);
-    assertEquals(PredicateLeaf.Type.STRING, leaf.getType());
-    assertEquals(PredicateLeaf.Operator.EQUALS, leaf.getOperator());
-    assertEquals("last_name", leaf.getColumnName());
-    assertEquals("smith", leaf.getLiteral());
-
-    assertEquals("(and leaf-0 leaf-1 leaf-2)",
-        sarg.getExpression().toString());
-    assertNoSharedNodes(sarg.getExpression(),
-        Sets.<ExpressionTree>newIdentityHashSet());
-  }
-
-  @Test
-  public void testExpression4() throws Exception {
-    /* id <> 12 and
-       first_name in ('john', 'sue') and
-       id in (34,50) */
-    String exprStr = "<?xml version=\"1.0\" encoding=\"UTF-8\"?> \n" +
-        "<java version=\"1.6.0_31\" class=\"java.beans.XMLDecoder\"> \n" +
-        " <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
-        "  <void property=\"children\"> \n" +
-        "   <object class=\"java.util.ArrayList\"> \n" +
-        "    <void method=\"add\"> \n" +
-        "     <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
-        "      <void property=\"children\"> \n" +
-        "       <object class=\"java.util.ArrayList\"> \n" +
-        "        <void method=\"add\"> \n" +
-        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
-        "          <void property=\"children\"> \n" +
-        "           <object class=\"java.util.ArrayList\"> \n" +
-        "            <void method=\"add\"> \n" +
-        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
-        "              <void property=\"column\"> \n" +
-        "               <string>id</string> \n" +
-        "              </void> \n" +
-        "              <void property=\"tabAlias\"> \n" +
-        "               <string>orc_people</string> \n" +
-        "              </void> \n" +
-        "              <void property=\"typeInfo\"> \n" +
-        "               <object id=\"PrimitiveTypeInfo0\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
-        "                <void property=\"typeName\"> \n" +
-        "                 <string>int</string> \n" +
-        "                </void> \n" +
-        "               </object> \n" +
-        "              </void> \n" +
-        "             </object> \n" +
-        "            </void> \n" +
-        "            <void method=\"add\"> \n" +
-        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
-        "              <void property=\"typeInfo\"> \n" +
-        "               <object idref=\"PrimitiveTypeInfo0\"/> \n" +
-        "              </void> \n" +
-        "              <void property=\"value\"> \n" +
-        "               <int>12</int> \n" +
-        "              </void> \n" +
-        "             </object> \n" +
-        "            </void> \n" +
-        "           </object> \n" +
-        "          </void> \n" +
-        "          <void property=\"genericUDF\"> \n" +
-        "           <object class=\"org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNotEqual\"/> \n" +
-        "          </void> \n" +
-        "          <void property=\"typeInfo\"> \n" +
-        "           <object id=\"PrimitiveTypeInfo1\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
-        "            <void property=\"typeName\"> \n" +
-        "             <string>boolean</string> \n" +
-        "            </void> \n" +
-        "           </object> \n" +
-        "          </void> \n" +
-        "         </object> \n" +
-        "        </void> \n" +
-        "        <void method=\"add\"> \n" +
-        "         <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc\"> \n" +
-        "          <void property=\"children\"> \n" +
-        "           <object class=\"java.util.ArrayList\"> \n" +
-        "            <void method=\"add\"> \n" +
-        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc\"> \n" +
-        "              <void property=\"column\"> \n" +
-        "               <string>first_name</string> \n" +
-        "              </void> \n" +
-        "              <void property=\"tabAlias\"> \n" +
-        "               <string>orc_people</string> \n" +
-        "              </void> \n" +
-        "              <void property=\"typeInfo\"> \n" +
-        "               <object id=\"PrimitiveTypeInfo2\" class=\"org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo\"> \n" +
-        "                <void property=\"typeName\"> \n" +
-        "                 <string>string</string> \n" +
-        "                </void> \n" +
-        "               </object> \n" +
-        "              </void> \n" +
-        "             </object> \n" +
-        "            </void> \n" +
-        "            <void method=\"add\"> \n" +
-        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
-        "              <void property=\"typeInfo\"> \n" +
-        "               <object idref=\"PrimitiveTypeInfo2\"/> \n" +
-        "              </void> \n" +
-        "              <void property=\"value\"> \n" +
-        "               <string>john</string> \n" +
-        "              </void> \n" +
-        "             </object> \n" +
-        "            </void> \n" +
-        "            <void method=\"add\"> \n" +
-        "             <object class=\"org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc\"> \n" +
-        "              <void property=\"typeInfo\"> \n" +
-        "               <object idref=\"PrimitiveTypeInfo2\"/> \n" +
-        "              </void> \n" +
-        "              <void property=\"value\"> \n" +
-        "               <string>sue</string> \n" +
-        "              </void> \n" +
-        "             </object> \n" +
-        "            </void> \n" +
-        "           </object> \n" +
-        "          </void> \n" +
-        "          <void property=\"genericUDF\"> \n" +
-        "           <object class=\"org.apache.hadoop.hive.ql.udf.generic.G

<TRUNCATED>

[19/43] hive git commit: HIVE-11333: ColumnPruner prunes columns of UnionOperator that should be kept (Pengcheng Xiong, via Jesus Camacho Rodriguez)

Posted by xu...@apache.org.
HIVE-11333: ColumnPruner prunes columns of UnionOperator that should be kept (Pengcheng Xiong, via 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/2773b9d9
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/2773b9d9
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/2773b9d9

Branch: refs/heads/spark
Commit: 2773b9d9fc9a1b16fcf22b4a976348fe591c6168
Parents: 393d690
Author: Pengcheng Xiong <px...@hortonworks.com>
Authored: Fri Jul 24 18:03:23 2015 +0200
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Fri Jul 24 18:03:41 2015 +0200

----------------------------------------------------------------------
 .../hadoop/hive/ql/optimizer/ColumnPruner.java  |   4 +
 .../hive/ql/optimizer/ColumnPrunerProcCtx.java  |  31 +-
 .../ql/optimizer/ColumnPrunerProcFactory.java   |  40 +++
 .../results/clientpositive/spark/union16.q.out  | 304 +++++++++-------
 .../results/clientpositive/spark/union2.q.out   |  28 +-
 .../results/clientpositive/spark/union9.q.out   |  40 ++-
 .../clientpositive/spark/union_view.q.out       |  24 ++
 .../results/clientpositive/tez/union2.q.out     |  28 +-
 .../results/clientpositive/tez/union9.q.out     |  40 ++-
 .../tez/vector_null_projection.q.out            |   4 +
 .../test/results/clientpositive/union16.q.out   | 354 +++++++++++--------
 ql/src/test/results/clientpositive/union2.q.out |  32 +-
 ql/src/test/results/clientpositive/union9.q.out |  46 +--
 .../results/clientpositive/union_view.q.out     |  24 ++
 .../clientpositive/vector_null_projection.q.out |   4 +
 15 files changed, 631 insertions(+), 372 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/2773b9d9/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPruner.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPruner.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPruner.java
index b8f5c71..9a45458 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPruner.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPruner.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
 import org.apache.hadoop.hive.ql.exec.ScriptOperator;
 import org.apache.hadoop.hive.ql.exec.SelectOperator;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
+import org.apache.hadoop.hive.ql.exec.UnionOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
 import org.apache.hadoop.hive.ql.lib.Dispatcher;
@@ -119,6 +120,9 @@ public class ColumnPruner implements Transform {
     opRules.put(new RuleRegExp("R12",
         LimitOperator.getOperatorName() + "%"),
         ColumnPrunerProcFactory.getLimitProc());
+    opRules.put(new RuleRegExp("R13",
+        UnionOperator.getOperatorName() + "%"),
+        ColumnPrunerProcFactory.getUnionProc());
     // The dispatcher fires the processor corresponding to the closest matching
     // rule and passes the context along
     Dispatcher disp = new DefaultRuleDispatcher(ColumnPrunerProcFactory

http://git-wip-us.apache.org/repos/asf/hive/blob/2773b9d9/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcCtx.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcCtx.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcCtx.java
index 692319e..c076d4e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcCtx.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcCtx.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.hive.ql.exec.CommonJoinOperator;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.RowSchema;
 import org.apache.hadoop.hive.ql.exec.SelectOperator;
+import org.apache.hadoop.hive.ql.exec.UnionOperator;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
@@ -47,10 +48,13 @@ public class ColumnPrunerProcCtx implements NodeProcessorCtx {
 
   private final Map<CommonJoinOperator, Map<Byte, List<String>>> joinPrunedColLists;
 
+  private final Map<UnionOperator, List<Integer>> unionPrunedColLists;
+
   public ColumnPrunerProcCtx(ParseContext pctx) {
     this.pctx = pctx;
     prunedColLists = new HashMap<Operator<? extends OperatorDesc>, List<String>>();
     joinPrunedColLists = new HashMap<CommonJoinOperator, Map<Byte, List<String>>>();
+    unionPrunedColLists = new HashMap<>();
   }
 
   public ParseContext getParseContext() {
@@ -61,6 +65,10 @@ public class ColumnPrunerProcCtx implements NodeProcessorCtx {
     return joinPrunedColLists;
   }
 
+  public Map<UnionOperator, List<Integer>> getUnionPrunedColLists() {
+    return unionPrunedColLists;
+  }
+
   /**
    * @return the prunedColLists
    */
@@ -90,10 +98,20 @@ public class ColumnPrunerProcCtx implements NodeProcessorCtx {
     }
     List<String> colList = null;
     for (Operator<? extends OperatorDesc> child : curOp.getChildOperators()) {
-      List<String> prunList;
+      List<String> prunList = null;
       if (child instanceof CommonJoinOperator) {
         int tag = child.getParentOperators().indexOf(curOp);
         prunList = joinPrunedColLists.get(child).get((byte) tag);
+      } else if (child instanceof UnionOperator) {
+        List<Integer> positions = unionPrunedColLists.get(child);
+        if (positions != null && positions.size() > 0) {
+          prunList = new ArrayList<>();
+          RowSchema oldRS = curOp.getSchema();
+          for (Integer pos : positions) {
+            ColumnInfo colInfo = oldRS.getSignature().get(pos);
+            prunList.add(colInfo.getInternalName());
+          }
+        }
       } else {
         prunList = prunedColLists.get(child);
       }
@@ -131,6 +149,17 @@ public class ColumnPrunerProcCtx implements NodeProcessorCtx {
     if (child instanceof CommonJoinOperator) {
       int tag = child.getParentOperators().indexOf(curOp);
       return joinPrunedColLists.get(child).get((byte) tag);
+    } else if (child instanceof UnionOperator) {
+      List<Integer> positions = unionPrunedColLists.get(child);
+      List<String> prunList = new ArrayList<>();
+      if (positions != null && positions.size() > 0) {
+        RowSchema oldRS = curOp.getSchema();
+        for (Integer pos : positions) {
+          ColumnInfo colInfo = oldRS.getSignature().get(pos);
+          prunList.add(colInfo.getInternalName());
+        }
+      }
+      return prunList;
     } else {
       return prunedColLists.get(child);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/2773b9d9/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java
index 26b83ca..ac4236c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ColumnPrunerProcFactory.java
@@ -47,6 +47,7 @@ import org.apache.hadoop.hive.ql.exec.RowSchema;
 import org.apache.hadoop.hive.ql.exec.SelectOperator;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.exec.UDTFOperator;
+import org.apache.hadoop.hive.ql.exec.UnionOperator;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.lib.Node;
 import org.apache.hadoop.hive.ql.lib.NodeProcessor;
@@ -68,6 +69,7 @@ import org.apache.hadoop.hive.ql.plan.ReduceSinkDesc;
 import org.apache.hadoop.hive.ql.plan.SelectDesc;
 import org.apache.hadoop.hive.ql.plan.TableDesc;
 import org.apache.hadoop.hive.ql.plan.TableScanDesc;
+import org.apache.hadoop.hive.ql.plan.UnionDesc;
 import org.apache.hadoop.hive.ql.plan.ptf.PTFExpressionDef;
 import org.apache.hadoop.hive.ql.plan.ptf.PTFInputDef;
 import org.apache.hadoop.hive.ql.plan.ptf.PartitionedTableFunctionDef;
@@ -955,6 +957,44 @@ public final class ColumnPrunerProcFactory {
     }
   }
 
+  /**
+   * The Factory method to get UnionProc class.
+   *
+   * @return UnionProc
+   */
+  public static ColumnPrunerUnionProc getUnionProc() {
+    return new ColumnPrunerUnionProc();
+  }
+
+  /**
+   * The Node Processor for Column Pruning on Union Operators.
+   */
+  public static class ColumnPrunerUnionProc implements NodeProcessor {
+    @Override
+    public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx ctx, Object... nodeOutputs)
+        throws SemanticException {
+      ColumnPrunerProcCtx cppCtx = (ColumnPrunerProcCtx) ctx;
+      UnionOperator op = (UnionOperator) nd;
+      List<String> childColLists = cppCtx.genColLists(op);
+      if (childColLists == null) {
+        return null;
+      }
+      RowSchema inputSchema = op.getSchema();
+      if (inputSchema != null) {
+        List<Integer> positions = new ArrayList<>();
+        RowSchema oldRS = op.getSchema();
+        for (int index = 0; index < oldRS.getSignature().size(); index++) {
+          ColumnInfo colInfo = oldRS.getSignature().get(index);
+          if (childColLists.contains(colInfo.getInternalName())) {
+            positions.add(index);
+          }
+        }
+        cppCtx.getUnionPrunedColLists().put(op, positions);
+      }
+      return null;
+    }
+  }
+
   private static void pruneOperator(NodeProcessorCtx ctx,
       Operator<? extends OperatorDesc> op,
       List<String> cols)

http://git-wip-us.apache.org/repos/asf/hive/blob/2773b9d9/ql/src/test/results/clientpositive/spark/union16.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union16.q.out b/ql/src/test/results/clientpositive/spark/union16.q.out
index 5e2c77b..6e45714 100644
--- a/ql/src/test/results/clientpositive/spark/union16.q.out
+++ b/ql/src/test/results/clientpositive/spark/union16.q.out
@@ -79,451 +79,501 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: src
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    expressions: value (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
                         aggregations: count(1)
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        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: COMPLETE
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col0 (type: bigint)
         Map 10 
             Map Operator Tree:
                 TableScan
                   alias: src
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    expressions: value (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
                         aggregations: count(1)
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        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: COMPLETE
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col0 (type: bigint)
         Map 11 
             Map Operator Tree:
                 TableScan
                   alias: src
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    expressions: value (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
                         aggregations: count(1)
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        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: COMPLETE
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col0 (type: bigint)
         Map 12 
             Map Operator Tree:
                 TableScan
                   alias: src
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    expressions: value (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
                         aggregations: count(1)
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        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: COMPLETE
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col0 (type: bigint)
         Map 13 
             Map Operator Tree:
                 TableScan
                   alias: src
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    expressions: value (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
                         aggregations: count(1)
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        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: COMPLETE
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col0 (type: bigint)
         Map 14 
             Map Operator Tree:
                 TableScan
                   alias: src
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    expressions: value (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
                         aggregations: count(1)
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        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: COMPLETE
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col0 (type: bigint)
         Map 15 
             Map Operator Tree:
                 TableScan
                   alias: src
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    expressions: value (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
                         aggregations: count(1)
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        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: COMPLETE
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col0 (type: bigint)
         Map 16 
             Map Operator Tree:
                 TableScan
                   alias: src
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    expressions: value (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
                         aggregations: count(1)
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        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: COMPLETE
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col0 (type: bigint)
         Map 17 
             Map Operator Tree:
                 TableScan
                   alias: src
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    expressions: value (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
                         aggregations: count(1)
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        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: COMPLETE
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col0 (type: bigint)
         Map 18 
             Map Operator Tree:
                 TableScan
                   alias: src
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    expressions: value (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
                         aggregations: count(1)
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        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: COMPLETE
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col0 (type: bigint)
         Map 19 
             Map Operator Tree:
                 TableScan
                   alias: src
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    expressions: value (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
                         aggregations: count(1)
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        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: COMPLETE
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col0 (type: bigint)
         Map 20 
             Map Operator Tree:
                 TableScan
                   alias: src
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    expressions: value (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
                         aggregations: count(1)
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        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: COMPLETE
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col0 (type: bigint)
         Map 21 
             Map Operator Tree:
                 TableScan
                   alias: src
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    expressions: value (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
                         aggregations: count(1)
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        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: COMPLETE
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col0 (type: bigint)
         Map 22 
             Map Operator Tree:
                 TableScan
                   alias: src
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    expressions: value (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
                         aggregations: count(1)
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        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: COMPLETE
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col0 (type: bigint)
         Map 23 
             Map Operator Tree:
                 TableScan
                   alias: src
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    expressions: value (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
                         aggregations: count(1)
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        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: COMPLETE
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col0 (type: bigint)
         Map 24 
             Map Operator Tree:
                 TableScan
                   alias: src
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    expressions: value (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
                         aggregations: count(1)
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        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: COMPLETE
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col0 (type: bigint)
         Map 25 
             Map Operator Tree:
                 TableScan
                   alias: src
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    expressions: value (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
                         aggregations: count(1)
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        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: COMPLETE
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col0 (type: bigint)
         Map 26 
             Map Operator Tree:
                 TableScan
                   alias: src
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    expressions: value (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
                         aggregations: count(1)
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        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: COMPLETE
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col0 (type: bigint)
         Map 3 
             Map Operator Tree:
                 TableScan
                   alias: src
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    expressions: value (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
                         aggregations: count(1)
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        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: COMPLETE
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col0 (type: bigint)
         Map 4 
             Map Operator Tree:
                 TableScan
                   alias: src
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    expressions: value (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
                         aggregations: count(1)
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        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: COMPLETE
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col0 (type: bigint)
         Map 5 
             Map Operator Tree:
                 TableScan
                   alias: src
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    expressions: value (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
                         aggregations: count(1)
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        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: COMPLETE
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col0 (type: bigint)
         Map 6 
             Map Operator Tree:
                 TableScan
                   alias: src
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    expressions: value (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
                         aggregations: count(1)
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        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: COMPLETE
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col0 (type: bigint)
         Map 7 
             Map Operator Tree:
                 TableScan
                   alias: src
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    expressions: value (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
                         aggregations: count(1)
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        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: COMPLETE
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col0 (type: bigint)
         Map 8 
             Map Operator Tree:
                 TableScan
                   alias: src
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    expressions: value (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
                         aggregations: count(1)
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        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: COMPLETE
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col0 (type: bigint)
         Map 9 
             Map Operator Tree:
                 TableScan
                   alias: src
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    expressions: value (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      Statistics: Num rows: 12500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                      Statistics: Num rows: 12500 Data size: 132800 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
                         aggregations: count(1)
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        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: COMPLETE
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col0 (type: bigint)
         Reducer 2 
             Reduce Operator Tree:
@@ -531,10 +581,10 @@ STAGE PLANS:
                 aggregations: count(VALUE._col0)
                 mode: mergepartial
                 outputColumnNames: _col0
-                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/2773b9d9/ql/src/test/results/clientpositive/spark/union2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union2.q.out b/ql/src/test/results/clientpositive/spark/union2.q.out
index e4afb1b..9e44bef 100644
--- a/ql/src/test/results/clientpositive/spark/union2.q.out
+++ b/ql/src/test/results/clientpositive/spark/union2.q.out
@@ -27,37 +27,41 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    expressions: value (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      Statistics: Num rows: 1000 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                      Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
                         aggregations: count(1)
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        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: COMPLETE
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col0 (type: bigint)
         Map 3 
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    expressions: value (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      Statistics: Num rows: 1000 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                      Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
                         aggregations: count(1)
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        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: COMPLETE
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col0 (type: bigint)
         Reducer 2 
             Reduce Operator Tree:
@@ -65,10 +69,10 @@ STAGE PLANS:
                 aggregations: count(VALUE._col0)
                 mode: mergepartial
                 outputColumnNames: _col0
-                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/2773b9d9/ql/src/test/results/clientpositive/spark/union9.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union9.q.out b/ql/src/test/results/clientpositive/spark/union9.q.out
index d420ef1..cb41d91 100644
--- a/ql/src/test/results/clientpositive/spark/union9.q.out
+++ b/ql/src/test/results/clientpositive/spark/union9.q.out
@@ -29,55 +29,61 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    expressions: value (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      Statistics: Num rows: 1500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                      Statistics: Num rows: 1500 Data size: 15936 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
                         aggregations: count(1)
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        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: COMPLETE
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col0 (type: bigint)
         Map 3 
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    expressions: value (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      Statistics: Num rows: 1500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                      Statistics: Num rows: 1500 Data size: 15936 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
                         aggregations: count(1)
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        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: COMPLETE
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col0 (type: bigint)
         Map 4 
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    expressions: value (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      Statistics: Num rows: 1500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                      Statistics: Num rows: 1500 Data size: 15936 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
                         aggregations: count(1)
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        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: COMPLETE
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col0 (type: bigint)
         Reducer 2 
             Reduce Operator Tree:
@@ -85,10 +91,10 @@ STAGE PLANS:
                 aggregations: count(VALUE._col0)
                 mode: mergepartial
                 outputColumnNames: _col0
-                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/2773b9d9/ql/src/test/results/clientpositive/spark/union_view.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union_view.q.out b/ql/src/test/results/clientpositive/spark/union_view.q.out
index 31e0786..8f19534 100644
--- a/ql/src/test/results/clientpositive/spark/union_view.q.out
+++ b/ql/src/test/results/clientpositive/spark/union_view.q.out
@@ -650,6 +650,8 @@ STAGE PLANS:
                   filterExpr: (ds = '1') (type: boolean)
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
+                    expressions: '1' (type: string)
+                    outputColumnNames: _col0
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -672,6 +674,8 @@ STAGE PLANS:
                     predicate: (ds = '1') (type: boolean)
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Select Operator
+                      expressions: '1' (type: string)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                       Select Operator
                         Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -694,6 +698,8 @@ STAGE PLANS:
                     predicate: (ds = '1') (type: boolean)
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Select Operator
+                      expressions: '1' (type: string)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                       Select Operator
                         Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -748,6 +754,8 @@ STAGE PLANS:
                     predicate: (ds = '2') (type: boolean)
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Select Operator
+                      expressions: '2' (type: string)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                       Select Operator
                         Statistics: Num rows: 1002 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
@@ -767,6 +775,8 @@ STAGE PLANS:
                   filterExpr: (ds = '2') (type: boolean)
                   Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
+                    expressions: '2' (type: string)
+                    outputColumnNames: _col0
                     Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       Statistics: Num rows: 1002 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
@@ -789,6 +799,8 @@ STAGE PLANS:
                     predicate: (ds = '2') (type: boolean)
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Select Operator
+                      expressions: '2' (type: string)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                       Select Operator
                         Statistics: Num rows: 1002 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
@@ -843,6 +855,8 @@ STAGE PLANS:
                     predicate: (ds = '3') (type: boolean)
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Select Operator
+                      expressions: '3' (type: string)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                       Select Operator
                         Statistics: Num rows: 1002 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
@@ -865,6 +879,8 @@ STAGE PLANS:
                     predicate: (ds = '3') (type: boolean)
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Select Operator
+                      expressions: '3' (type: string)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                       Select Operator
                         Statistics: Num rows: 1002 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
@@ -884,6 +900,8 @@ STAGE PLANS:
                   filterExpr: (ds = '3') (type: boolean)
                   Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
+                    expressions: '3' (type: string)
+                    outputColumnNames: _col0
                     Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       Statistics: Num rows: 1002 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
@@ -1030,6 +1048,8 @@ STAGE PLANS:
                     predicate: (ds = '4') (type: boolean)
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Select Operator
+                      expressions: '4' (type: string)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                       Select Operator
                         Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -1052,6 +1072,8 @@ STAGE PLANS:
                     predicate: (ds = '4') (type: boolean)
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Select Operator
+                      expressions: '4' (type: string)
+                      outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                       Select Operator
                         Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -1071,6 +1093,8 @@ STAGE PLANS:
                   filterExpr: (ds = '4') (type: boolean)
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
+                    expressions: '4' (type: string)
+                    outputColumnNames: _col0
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/2773b9d9/ql/src/test/results/clientpositive/tez/union2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/union2.q.out b/ql/src/test/results/clientpositive/tez/union2.q.out
index c127089..672faf2 100644
--- a/ql/src/test/results/clientpositive/tez/union2.q.out
+++ b/ql/src/test/results/clientpositive/tez/union2.q.out
@@ -29,37 +29,41 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    expressions: value (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      Statistics: Num rows: 1000 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                      Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
                         aggregations: count(1)
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        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: COMPLETE
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col0 (type: bigint)
         Map 4 
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    expressions: value (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      Statistics: Num rows: 1000 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                      Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
                         aggregations: count(1)
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        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: COMPLETE
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col0 (type: bigint)
         Reducer 3 
             Reduce Operator Tree:
@@ -67,10 +71,10 @@ STAGE PLANS:
                 aggregations: count(VALUE._col0)
                 mode: mergepartial
                 outputColumnNames: _col0
-                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/2773b9d9/ql/src/test/results/clientpositive/tez/union9.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/union9.q.out b/ql/src/test/results/clientpositive/tez/union9.q.out
index ee26daf..ca3e532 100644
--- a/ql/src/test/results/clientpositive/tez/union9.q.out
+++ b/ql/src/test/results/clientpositive/tez/union9.q.out
@@ -32,55 +32,61 @@ STAGE PLANS:
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    expressions: value (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      Statistics: Num rows: 1500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                      Statistics: Num rows: 1500 Data size: 15936 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
                         aggregations: count(1)
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        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: COMPLETE
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col0 (type: bigint)
         Map 4 
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    expressions: value (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      Statistics: Num rows: 1500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                      Statistics: Num rows: 1500 Data size: 15936 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
                         aggregations: count(1)
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        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: COMPLETE
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col0 (type: bigint)
         Map 5 
             Map Operator Tree:
                 TableScan
                   alias: s1
-                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
-                    Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                    expressions: value (type: string)
+                    outputColumnNames: _col0
+                    Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
-                      Statistics: Num rows: 1500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+                      Statistics: Num rows: 1500 Data size: 15936 Basic stats: COMPLETE Column stats: NONE
                       Group By Operator
                         aggregations: count(1)
                         mode: hash
                         outputColumnNames: _col0
-                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                        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: COMPLETE
+                          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           value expressions: _col0 (type: bigint)
         Reducer 3 
             Reduce Operator Tree:
@@ -88,10 +94,10 @@ STAGE PLANS:
                 aggregations: count(VALUE._col0)
                 mode: mergepartial
                 outputColumnNames: _col0
-                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
-                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat

http://git-wip-us.apache.org/repos/asf/hive/blob/2773b9d9/ql/src/test/results/clientpositive/tez/vector_null_projection.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vector_null_projection.q.out b/ql/src/test/results/clientpositive/tez/vector_null_projection.q.out
index 88587e9..79802da 100644
--- a/ql/src/test/results/clientpositive/tez/vector_null_projection.q.out
+++ b/ql/src/test/results/clientpositive/tez/vector_null_projection.q.out
@@ -110,6 +110,8 @@ STAGE PLANS:
                   alias: a
                   Statistics: Num rows: 1 Data size: 87 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
+                    expressions: null (type: void)
+                    outputColumnNames: _col0
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
                     Select Operator
                       Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
@@ -129,6 +131,8 @@ STAGE PLANS:
                   alias: b
                   Statistics: Num rows: 1 Data size: 87 Basic stats: COMPLETE Column stats: COMPLETE
                   Select Operator
+                    expressions: null (type: void)
+                    outputColumnNames: _col0
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
                     Select Operator
                       Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE