You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ga...@apache.org on 2015/07/22 22:47:29 UTC

[01/50] [abbrv] hive git commit: HIVE-11240 Change value type from int to long for HiveConf.ConfVars.METASTORESERVERMAXMESSAGESIZE(Ferdinand Xu, reviewed by Xuefu Zhang)

Repository: hive
Updated Branches:
  refs/heads/hbase-metastore a310524c4 -> 61db7b80c


HIVE-11240 Change value type from int to long for HiveConf.ConfVars.METASTORESERVERMAXMESSAGESIZE(Ferdinand Xu, reviewed by Xuefu Zhang)


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

Branch: refs/heads/hbase-metastore
Commit: 21aecbcfff21d33f1785221be73f880d728632fa
Parents: 5363af9
Author: Ferdinand Xu <ch...@intel.com>
Authored: Tue Jul 14 02:45:54 2015 -0400
Committer: Ferdinand Xu <ch...@intel.com>
Committed: Tue Jul 14 02:45:54 2015 -0400

----------------------------------------------------------------------
 .../src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/21aecbcf/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 920e762..2ef5aa0 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -5995,7 +5995,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       // Server will create new threads up to max as necessary. After an idle
       // period, it will destroy threads to keep the number of threads in the
       // pool to min.
-      int maxMessageSize = conf.getIntVar(HiveConf.ConfVars.METASTORESERVERMAXMESSAGESIZE);
+      long maxMessageSize = conf.getIntVar(HiveConf.ConfVars.METASTORESERVERMAXMESSAGESIZE);
       int minWorkerThreads = conf.getIntVar(HiveConf.ConfVars.METASTORESERVERMINTHREADS);
       int maxWorkerThreads = conf.getIntVar(HiveConf.ConfVars.METASTORESERVERMAXTHREADS);
       boolean tcpKeepAlive = conf.getBoolVar(HiveConf.ConfVars.METASTORE_TCP_KEEP_ALIVE);


[34/50] [abbrv] hive git commit: HIVE-11262: Skip MapJoin processing if the join hash table is empty (Jason Dere, reviewed by Vikram Dixit)

Posted by ga...@apache.org.
HIVE-11262: Skip MapJoin processing if the join hash table is empty (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/941610f2
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/941610f2
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/941610f2

Branch: refs/heads/hbase-metastore
Commit: 941610f2a343273d448e5344ee759c3cc7032863
Parents: a5cc034
Author: Jason Dere <jd...@hortonworks.com>
Authored: Mon Jul 20 10:47:37 2015 -0700
Committer: Jason Dere <jd...@hortonworks.com>
Committed: Mon Jul 20 10:47:37 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hive/ql/exec/MapJoinOperator.java    | 59 ++++++++++++++++++--
 .../persistence/HybridHashTableContainer.java   | 27 +++++++++
 .../persistence/MapJoinBytesTableContainer.java |  5 ++
 .../exec/persistence/MapJoinTableContainer.java |  5 ++
 .../fast/VectorMapJoinFastHashTable.java        |  5 ++
 .../fast/VectorMapJoinFastTableContainer.java   |  5 ++
 .../hashtable/VectorMapJoinHashTable.java       |  4 ++
 .../VectorMapJoinOptimizedHashTable.java        |  4 ++
 8 files changed, 108 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/941610f2/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 15cafdd..a40f0a9 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
@@ -55,6 +55,8 @@ import org.apache.hadoop.hive.ql.exec.spark.SparkUtilities;
 import org.apache.hadoop.hive.ql.io.HiveKey;
 import org.apache.hadoop.hive.ql.log.PerfLogger;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.JoinCondDesc;
+import org.apache.hadoop.hive.ql.plan.JoinDesc;
 import org.apache.hadoop.hive.ql.plan.MapJoinDesc;
 import org.apache.hadoop.hive.ql.plan.TableDesc;
 import org.apache.hadoop.hive.ql.plan.api.OperatorType;
@@ -176,8 +178,7 @@ public class MapJoinOperator extends AbstractMapJoinOperator<MapJoinDesc> implem
                 }
               });
       result.add(future);
-    } else if (mapContext == null || mapContext.getLocalWork() == null
-        || mapContext.getLocalWork().getInputFileChangeSensitive() == false) {
+    } else if (!isInputFileChangeSensitive(mapContext)) {
       loadHashTable(mapContext, mrContext);
       hashTblInitedOnce = true;
     }
@@ -276,9 +277,7 @@ public class MapJoinOperator extends AbstractMapJoinOperator<MapJoinDesc> implem
       ExecMapperContext mapContext, MapredContext mrContext) throws HiveException {
     loadCalled = true;
 
-    if (this.hashTblInitedOnce
-        && ((mapContext == null) || (mapContext.getLocalWork() == null) || (mapContext
-            .getLocalWork().getInputFileChangeSensitive() == false))) {
+    if (canSkipReload(mapContext)) {
       // no need to reload
       return new ImmutablePair<MapJoinTableContainer[], MapJoinTableContainerSerDe[]>(
           mapJoinTables, mapJoinTableSerdes);
@@ -306,6 +305,11 @@ public class MapJoinOperator extends AbstractMapJoinOperator<MapJoinDesc> implem
 
     perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.LOAD_HASHTABLE);
 
+    if (canSkipJoinProcessing(mapContext)) {
+      LOG.info("Skipping big table join processing for " + this.toString());
+      this.setDone(true);
+    }
+
     return pair;
   }
 
@@ -611,7 +615,7 @@ public class MapJoinOperator extends AbstractMapJoinOperator<MapJoinDesc> implem
     }
 
     container.setTotalInMemRowCount(container.getTotalInMemRowCount()
-        + restoredHashMap.getNumValues() + kvContainer.size());
+        + restoredHashMap.getNumValues());
     kvContainer.clear();
 
     spilledMapJoinTables[pos] = new MapJoinBytesTableContainer(restoredHashMap);
@@ -656,4 +660,47 @@ public class MapJoinOperator extends AbstractMapJoinOperator<MapJoinDesc> implem
   public OperatorType getType() {
     return OperatorType.MAPJOIN;
   }
+
+  protected boolean isInputFileChangeSensitive(ExecMapperContext mapContext) {
+    return !(mapContext == null
+        || mapContext.getLocalWork() == null
+        || mapContext.getLocalWork().getInputFileChangeSensitive() == false);
+  }
+
+  protected boolean canSkipReload(ExecMapperContext mapContext) {
+    return (this.hashTblInitedOnce && !isInputFileChangeSensitive(mapContext));
+  }
+
+  // If the loaded hash table is empty, for some conditions we can skip processing the big table rows.
+  protected boolean canSkipJoinProcessing(ExecMapperContext mapContext) {
+    if (!canSkipReload(mapContext)) {
+      return false;
+    }
+
+    JoinCondDesc[] joinConds = getConf().getConds();
+    if (joinConds.length > 0) {
+      for (JoinCondDesc joinCond : joinConds) {
+        if (joinCond.getType() != JoinDesc.INNER_JOIN) {
+          return false;
+        }
+      }
+    } else {
+      return false;
+    }
+
+    boolean skipJoinProcessing = false;
+    for (int idx = 0; idx < mapJoinTables.length; ++idx) {
+      if (idx == getConf().getPosBigTable()) {
+        continue;
+      }
+      MapJoinTableContainer mapJoinTable = mapJoinTables[idx];
+      if (mapJoinTable.size() == 0) {
+        // If any table is empty, an inner join involving the tables should yield 0 rows.
+        LOG.info("Hash table number " + idx + " is empty");
+        skipJoinProcessing = true;
+        break;
+      }
+    }
+    return skipJoinProcessing;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/941610f2/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HybridHashTableContainer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HybridHashTableContainer.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HybridHashTableContainer.java
index e338a31..0a6461f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HybridHashTableContainer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HybridHashTableContainer.java
@@ -110,6 +110,7 @@ public class HybridHashTableContainer
     int threshold;                          // Used to create an empty BytesBytesMultiHashMap
     float loadFactor;                       // Same as above
     int wbSize;                             // Same as above
+    int rowsOnDisk;                         // How many rows saved to the on-disk hashmap (if on disk)
 
     /* It may happen that there's not enough memory to instantiate a hashmap for the partition.
      * In that case, we don't create the hashmap, but pretend the hashmap is directly "spilled".
@@ -149,6 +150,10 @@ public class HybridHashTableContainer
           restoredHashMap.expandAndRehashToTarget(initialCapacity);
         }
 
+        // some bookkeeping
+        rowsOnDisk = 0;
+        hashMapOnDisk = false;
+
         input.close();
         inputStream.close();
         Files.delete(hashMapLocalPath);
@@ -197,6 +202,8 @@ public class HybridHashTableContainer
         } catch (Throwable ignored) {
         }
         hashMapLocalPath = null;
+        rowsOnDisk = 0;
+        hashMapOnDisk = false;
       }
 
       if (sidefileKVContainer != null) {
@@ -214,6 +221,16 @@ public class HybridHashTableContainer
         matchfileRowBytesContainer = null;
       }
     }
+
+    public int size() {
+      if (isHashMapOnDisk()) {
+        // Rows are in a combination of the on-disk hashmap and the sidefile
+        return rowsOnDisk + (sidefileKVContainer != null ? sidefileKVContainer.size() : 0);
+      } else {
+        // All rows should be in the in-memory hashmap
+        return hashMap.size();
+      }
+    }
   }
 
   public HybridHashTableContainer(Configuration hconf, long keyCount, long memoryAvailable,
@@ -507,6 +524,7 @@ public class HybridHashTableContainer
     memoryUsed -= memFreed;
     LOG.info("Memory usage after spilling: " + memoryUsed);
 
+    partition.rowsOnDisk = inMemRowCount;
     totalInMemRowCount -= inMemRowCount;
     partition.hashMap.clear();
     return memFreed;
@@ -959,4 +977,13 @@ public class HybridHashTableContainer
         numPartitionsInMem + " partitions in memory have been processed; " +
         numPartitionsOnDisk + " partitions have been spilled to disk and will be processed next.");
   }
+
+  @Override
+  public int size() {
+    int totalSize = 0;
+    for (HashPartition hashPartition : hashPartitions) {
+      totalSize += hashPartition.size();
+    }
+    return totalSize;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/941610f2/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinBytesTableContainer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinBytesTableContainer.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinBytesTableContainer.java
index 83a1521..5df8e2b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinBytesTableContainer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinBytesTableContainer.java
@@ -736,4 +736,9 @@ public class MapJoinBytesTableContainer
   public boolean hasSpill() {
     return false;
   }
+
+  @Override
+  public int size() {
+    return hashMap.size();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/941610f2/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinTableContainer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinTableContainer.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinTableContainer.java
index 9d8cbcb..869aefd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinTableContainer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinTableContainer.java
@@ -106,4 +106,9 @@ public interface MapJoinTableContainer {
    * This is only applicable for HybridHashTableContainer.
    */
   boolean hasSpill();
+
+  /**
+   * Return the size of the hash table
+   */
+  int size();
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/941610f2/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastHashTable.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastHashTable.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastHashTable.java
index fbe6b4c..666d666 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastHashTable.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastHashTable.java
@@ -65,4 +65,9 @@ public abstract class VectorMapJoinFastHashTable implements VectorMapJoinHashTab
     this.loadFactor = loadFactor;
     this.writeBuffersSize = writeBuffersSize;
   }
+
+  @Override
+  public int size() {
+    return keysAssigned;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/941610f2/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastTableContainer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastTableContainer.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastTableContainer.java
index 4b1d6f6..f2080f4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastTableContainer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VectorMapJoinFastTableContainer.java
@@ -213,6 +213,11 @@ public class VectorMapJoinFastTableContainer implements VectorMapJoinTableContai
     return false;
   }
 
+  @Override
+  public int size() {
+    return VectorMapJoinFastHashTable.size();
+  }
+
   /*
   @Override
   public com.esotericsoftware.kryo.io.Output getHybridBigTableSpillOutput(int partitionId) {

http://git-wip-us.apache.org/repos/asf/hive/blob/941610f2/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/hashtable/VectorMapJoinHashTable.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/hashtable/VectorMapJoinHashTable.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/hashtable/VectorMapJoinHashTable.java
index 7e219ec..c7e585c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/hashtable/VectorMapJoinHashTable.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/hashtable/VectorMapJoinHashTable.java
@@ -40,4 +40,8 @@ public interface VectorMapJoinHashTable {
   void putRow(BytesWritable currentKey, BytesWritable currentValue)
       throws SerDeException, HiveException, IOException;
 
+  /**
+   * Get hash table size
+   */
+  int size();
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/941610f2/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/optimized/VectorMapJoinOptimizedHashTable.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/optimized/VectorMapJoinOptimizedHashTable.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/optimized/VectorMapJoinOptimizedHashTable.java
index a2d4e4c..b2b86d5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/optimized/VectorMapJoinOptimizedHashTable.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/optimized/VectorMapJoinOptimizedHashTable.java
@@ -92,4 +92,8 @@ public abstract class VectorMapJoinOptimizedHashTable implements VectorMapJoinHa
     adapatorDirectAccess = (ReusableGetAdaptorDirectAccess) hashMapRowGetter;
   }
 
+  @Override
+  public int size() {
+    return originalTableContainer.size();
+  }
 }


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

Posted by ga...@apache.org.
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/hbase-metastore
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());
+    }
+  }
+
+}


[27/50] [abbrv] hive git commit: HIVE-11279 : Hive should emit lineage information in json compact format (Lenni Kuff via Szehon)

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/2b1f03e3/ql/src/test/results/clientpositive/lineage3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/lineage3.q.out b/ql/src/test/results/clientpositive/lineage3.q.out
index 1afe92d..07b5f31 100644
--- a/ql/src/test/results/clientpositive/lineage3.q.out
+++ b/ql/src/test/results/clientpositive/lineage3.q.out
@@ -8,35 +8,7 @@ PREHOOK: type: CREATETABLE_AS_SELECT
 PREHOOK: Input: default@src1
 PREHOOK: Output: database:default
 PREHOOK: Output: default@t
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "761b3a1f405d8e719d3f0c9147b57a23",
-  "queryText": "create table t as\nselect * from\n  (select * from\n     (select key from src1 limit 1) v1) v2",
-  "edges": [
-    {
-      "sources": [
-        1
-      ],
-      "targets": [
-        0
-      ],
-      "edgeType": "PROJECTION"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "default.t.key"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.key"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"761b3a1f405d8e719d3f0c9147b57a23","queryText":"create table t as\nselect * from\n  (select * from\n     (select key from src1 limit 1) v1) v2","edges":[{"sources":[1],"targets":[0],"edgeType":"PROJECTION"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.t.key"},{"id":1,"vertexType":"COLUMN","vertexId":"default.src1.key"}]}
 PREHOOK: query: drop table if exists dest_l1
 PREHOOK: type: DROPTABLE
 PREHOOK: query: create table dest_l1(a int, b varchar(128))
@@ -51,66 +23,7 @@ where cint is not null and cint < 0 order by cint, cs limit 5
 PREHOOK: type: QUERY
 PREHOOK: Input: default@alltypesorc
 PREHOOK: Output: default@dest_l1@ds=today
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "2b5891d094ff74e23ec6acf5b4990f45",
-  "queryText": "insert into table dest_l1 partition (ds='today')\nselect cint, cast(cstring1 as varchar(128)) as cs\nfrom alltypesorc\nwhere cint is not null and cint < 0 order by cint, cs limit 5",
-  "edges": [
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        3
-      ],
-      "targets": [
-        1
-      ],
-      "expression": "CAST( alltypesorc.cstring1 AS varchar(128))",
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0,
-        1
-      ],
-      "expression": "(alltypesorc.cint is not null and (alltypesorc.cint < 0))",
-      "edgeType": "PREDICATE"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "cint"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "cs"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cint"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cstring1"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"2b5891d094ff74e23ec6acf5b4990f45","queryText":"insert into table dest_l1 partition (ds='today')\nselect cint, cast(cstring1 as varchar(128)) as cs\nfrom alltypesorc\nwhere cint is not null and cint < 0 order by cint, cs limit 5","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"expression":"CAST( alltypesorc.cstring1 AS varchar(128))","edgeType":"PROJECTION"},{"sources":[2],"targets":[0,1],"expression":"(alltypesorc.cint is not null and (alltypesorc.cint < 0))","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"cint"},{"id":1,"vertexType":"COLUMN","vertexId":"cs"},{"id":2,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"},{"id":3,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cstring1"}]}
 PREHOOK: query: insert into table dest_l1 partition (ds='tomorrow')
 select min(cint), cast(min(cstring1) as varchar(128)) as cs
 from alltypesorc
@@ -120,215 +33,13 @@ having min(cbigint) > 10
 PREHOOK: type: QUERY
 PREHOOK: Input: default@alltypesorc
 PREHOOK: Output: default@dest_l1@ds=tomorrow
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "4ad6338a8abfe3fe0342198fcbd1f11d",
-  "queryText": "insert into table dest_l1 partition (ds='tomorrow')\nselect min(cint), cast(min(cstring1) as varchar(128)) as cs\nfrom alltypesorc\nwhere cint is not null and cboolean1 = true\ngroup by csmallint\nhaving min(cbigint) > 10",
-  "edges": [
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0
-      ],
-      "expression": "min(default.alltypesorc.cint)",
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        3
-      ],
-      "targets": [
-        1
-      ],
-      "expression": "CAST( min(default.alltypesorc.cstring1) AS varchar(128))",
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        2,
-        4
-      ],
-      "targets": [
-        0,
-        1
-      ],
-      "expression": "(alltypesorc.cint is not null and (alltypesorc.cboolean1 = true))",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [
-        5
-      ],
-      "targets": [
-        0,
-        1
-      ],
-      "expression": "(min(default.alltypesorc.cbigint) > 10)",
-      "edgeType": "PREDICATE"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "c0"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "cs"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cint"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cstring1"
-    },
-    {
-      "id": 4,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cboolean1"
-    },
-    {
-      "id": 5,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cbigint"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"4ad6338a8abfe3fe0342198fcbd1f11d","queryText":"insert into table dest_l1 partition (ds='tomorrow')\nselect min(cint), cast(min(cstring1) as varchar(128)) as cs\nfrom alltypesorc\nwhere cint is not null and cboolean1 = true\ngroup by csmallint\nhaving min(cbigint) > 10","edges":[{"sources":[2],"targets":[0],"expression":"min(default.alltypesorc.cint)","edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"expression":"CAST( min(default.alltypesorc.cstring1) AS varchar(128))","edgeType":"PROJECTION"},{"sources":[2,4],"targets":[0,1],"expression":"(alltypesorc.cint is not null and (alltypesorc.cboolean1 = true))","edgeType":"PREDICATE"},{"sources":[5],"targets":[0,1],"expression":"(min(default.alltypesorc.cbigint) > 10)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"c0"},{"id":1,"vertexType":"COLUMN","vertexId":"cs"},{"id":2,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"},{"id":3,"vertexType":"COLUMN",
 "vertexId":"default.alltypesorc.cstring1"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean1"},{"id":5,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cbigint"}]}
 PREHOOK: query: select cint, rank() over(order by cint) from alltypesorc
 where cint > 10 and cint < 10000 limit 10
 PREHOOK: type: QUERY
 PREHOOK: Input: default@alltypesorc
 #### A masked pattern was here ####
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "351b08ec58591554ec10a6ded68ef25f",
-  "queryText": "select cint, rank() over(order by cint) from alltypesorc\nwhere cint > 10 and cint < 10000 limit 10",
-  "edges": [
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        3,
-        4,
-        2,
-        5,
-        6,
-        7,
-        8,
-        9,
-        10,
-        11,
-        12,
-        13
-      ],
-      "targets": [
-        1
-      ],
-      "expression": "(tok_function rank (tok_windowspec (tok_partitioningspec (tok_distributeby 0) (tok_orderby (tok_tabsortcolnameasc (. (tok_table_or_col alltypesorc) cint)))) (tok_windowrange (preceding 2147483647) (following 2147483647))))",
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0,
-        1
-      ],
-      "expression": "((alltypesorc.cint > 10) and (alltypesorc.cint < 10000))",
-      "edgeType": "PREDICATE"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "cint"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "c1"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cint"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.ctinyint"
-    },
-    {
-      "id": 4,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.csmallint"
-    },
-    {
-      "id": 5,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cbigint"
-    },
-    {
-      "id": 6,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cfloat"
-    },
-    {
-      "id": 7,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cdouble"
-    },
-    {
-      "id": 8,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cstring1"
-    },
-    {
-      "id": 9,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cstring2"
-    },
-    {
-      "id": 10,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.ctimestamp1"
-    },
-    {
-      "id": 11,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.ctimestamp2"
-    },
-    {
-      "id": 12,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cboolean1"
-    },
-    {
-      "id": 13,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cboolean2"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"351b08ec58591554ec10a6ded68ef25f","queryText":"select cint, rank() over(order by cint) from alltypesorc\nwhere cint > 10 and cint < 10000 limit 10","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3,4,2,5,6,7,8,9,10,11,12,13],"targets":[1],"expression":"(tok_function rank (tok_windowspec (tok_partitioningspec (tok_distributeby 0) (tok_orderby (tok_tabsortcolnameasc (. (tok_table_or_col alltypesorc) cint)))) (tok_windowrange (preceding 2147483647) (following 2147483647))))","edgeType":"PROJECTION"},{"sources":[2],"targets":[0,1],"expression":"((alltypesorc.cint > 10) and (alltypesorc.cint < 10000))","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"cint"},{"id":1,"vertexType":"COLUMN","vertexId":"c1"},{"id":2,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"},{"id":3,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"},{"id":4,"vertexType":"COLUMN","vertexId":"default.a
 lltypesorc.csmallint"},{"id":5,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cbigint"},{"id":6,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cfloat"},{"id":7,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cdouble"},{"id":8,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cstring1"},{"id":9,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cstring2"},{"id":10,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctimestamp1"},{"id":11,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctimestamp2"},{"id":12,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean1"},{"id":13,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean2"}]}
 762	1
 762	1
 762	1
@@ -347,95 +58,7 @@ order by a.ctinyint, a.cint
 PREHOOK: type: QUERY
 PREHOOK: Input: default@alltypesorc
 #### A masked pattern was here ####
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "40c3faa7abd1cdb7f12c1047a8a1d2ce",
-  "queryText": "select a.ctinyint, a.cint, count(a.cdouble)\n  over(partition by a.ctinyint order by a.cint desc\n    rows between 1 preceding and 1 following)\nfrom alltypesorc a inner join alltypesorc b on a.cint = b.cbigint\norder by a.ctinyint, a.cint",
-  "edges": [
-    {
-      "sources": [
-        3
-      ],
-      "targets": [
-        0
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        4
-      ],
-      "targets": [
-        1
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        3,
-        4,
-        5,
-        6
-      ],
-      "targets": [
-        2
-      ],
-      "expression": "(tok_function count (. (tok_table_or_col $hdt$_0) cdouble) (tok_windowspec (tok_partitioningspec (tok_distributeby (. (tok_table_or_col $hdt$_0) ctinyint)) (tok_orderby (tok_tabsortcolnamedesc (. (tok_table_or_col $hdt$_0) cint)))) (tok_windowrange (preceding 1) (following 1))))",
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        4,
-        6
-      ],
-      "targets": [
-        0,
-        1,
-        2
-      ],
-      "expression": "(UDFToLong(a.cint) = a.cbigint)",
-      "edgeType": "PREDICATE"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "a.ctinyint"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "a.cint"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "c2"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.ctinyint"
-    },
-    {
-      "id": 4,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cint"
-    },
-    {
-      "id": 5,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cdouble"
-    },
-    {
-      "id": 6,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cbigint"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"40c3faa7abd1cdb7f12c1047a8a1d2ce","queryText":"select a.ctinyint, a.cint, count(a.cdouble)\n  over(partition by a.ctinyint order by a.cint desc\n    rows between 1 preceding and 1 following)\nfrom alltypesorc a inner join alltypesorc b on a.cint = b.cbigint\norder by a.ctinyint, a.cint","edges":[{"sources":[3],"targets":[0],"edgeType":"PROJECTION"},{"sources":[4],"targets":[1],"edgeType":"PROJECTION"},{"sources":[3,4,5,6],"targets":[2],"expression":"(tok_function count (. (tok_table_or_col $hdt$_0) cdouble) (tok_windowspec (tok_partitioningspec (tok_distributeby (. (tok_table_or_col $hdt$_0) ctinyint)) (tok_orderby (tok_tabsortcolnamedesc (. (tok_table_or_col $hdt$_0) cint)))) (tok_windowrange (preceding 1) (following 1))))","edgeType":"PROJECTION"},{"sources":[4,6],"targets":[0,1,2],"expression":"(UDFToLong(a.cint) = a.cbigint)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"a.ctinyint"},{"id":1,"vertexType":"CO
 LUMN","vertexId":"a.cint"},{"id":2,"vertexType":"COLUMN","vertexId":"c2"},{"id":3,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"},{"id":5,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cdouble"},{"id":6,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cbigint"}]}
 PREHOOK: query: with v2 as
   (select cdouble, count(cint) over() a,
     sum(cint + cbigint) over(partition by cboolean1) b
@@ -445,194 +68,7 @@ order by 1, 2, 3 limit 5
 PREHOOK: type: QUERY
 PREHOOK: Input: default@alltypesorc
 #### A masked pattern was here ####
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "a1487a49aee6bd1e81b75185b21cfb54",
-  "queryText": "with v2 as\n  (select cdouble, count(cint) over() a,\n    sum(cint + cbigint) over(partition by cboolean1) b\n    from (select * from alltypesorc) v1)\nselect cdouble, a, b, a + b, cdouble + a from v2\norder by 1, 2, 3 limit 5",
-  "edges": [
-    {
-      "sources": [
-        5
-      ],
-      "targets": [
-        0
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        6,
-        7,
-        8,
-        9,
-        10,
-        5,
-        11,
-        12,
-        13,
-        14,
-        15,
-        16
-      ],
-      "targets": [
-        1
-      ],
-      "expression": "(tok_function count (. (tok_table_or_col alltypesorc) cint) (tok_windowspec (tok_partitioningspec (tok_distributeby 0) (tok_orderby (tok_tabsortcolnameasc 0))) (tok_windowrange (preceding 2147483647) (following 2147483647))))",
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        6,
-        7,
-        8,
-        9,
-        10,
-        5,
-        11,
-        12,
-        13,
-        14,
-        15,
-        16
-      ],
-      "targets": [
-        2
-      ],
-      "expression": "(tok_function sum (+ (tok_function tok_bigint (. (tok_table_or_col alltypesorc) cint)) (. (tok_table_or_col alltypesorc) cbigint)) (tok_windowspec (tok_partitioningspec (tok_distributeby (. (tok_table_or_col alltypesorc) cboolean1)) (tok_orderby (tok_tabsortcolnameasc (. (tok_table_or_col alltypesorc) cboolean1)))) (tok_windowrange (preceding 2147483647) (following 2147483647))))",
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        6,
-        7,
-        8,
-        9,
-        10,
-        5,
-        11,
-        12,
-        13,
-        14,
-        15,
-        16
-      ],
-      "targets": [
-        3
-      ],
-      "expression": "((tok_function count (. (tok_table_or_col alltypesorc) cint) (tok_windowspec (tok_partitioningspec (tok_distributeby 0) (tok_orderby (tok_tabsortcolnameasc 0))) (tok_windowrange (preceding 2147483647) (following 2147483647)))) + (tok_function sum (+ (tok_function tok_bigint (. (tok_table_or_col alltypesorc) cint)) (. (tok_table_or_col alltypesorc) cbigint)) (tok_windowspec (tok_partitioningspec (tok_distributeby (. (tok_table_or_col alltypesorc) cboolean1)) (tok_orderby (tok_tabsortcolnameasc (. (tok_table_or_col alltypesorc) cboolean1)))) (tok_windowrange (preceding 2147483647) (following 2147483647)))))",
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        5,
-        6,
-        7,
-        8,
-        9,
-        10,
-        11,
-        12,
-        13,
-        14,
-        15,
-        16
-      ],
-      "targets": [
-        4
-      ],
-      "expression": "(alltypesorc._col6 + UDFToDouble((tok_function count (. (tok_table_or_col alltypesorc) cint) (tok_windowspec (tok_partitioningspec (tok_distributeby 0) (tok_orderby (tok_tabsortcolnameasc 0))) (tok_windowrange (preceding 2147483647) (following 2147483647))))))",
-      "edgeType": "PROJECTION"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "cdouble"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "a"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "b"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "c3"
-    },
-    {
-      "id": 4,
-      "vertexType": "COLUMN",
-      "vertexId": "c4"
-    },
-    {
-      "id": 5,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cdouble"
-    },
-    {
-      "id": 6,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.ctinyint"
-    },
-    {
-      "id": 7,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.csmallint"
-    },
-    {
-      "id": 8,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cint"
-    },
-    {
-      "id": 9,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cbigint"
-    },
-    {
-      "id": 10,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cfloat"
-    },
-    {
-      "id": 11,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cstring1"
-    },
-    {
-      "id": 12,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cstring2"
-    },
-    {
-      "id": 13,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.ctimestamp1"
-    },
-    {
-      "id": 14,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.ctimestamp2"
-    },
-    {
-      "id": 15,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cboolean1"
-    },
-    {
-      "id": 16,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cboolean2"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"a1487a49aee6bd1e81b75185b21cfb54","queryText":"with v2 as\n  (select cdouble, count(cint) over() a,\n    sum(cint + cbigint) over(partition by cboolean1) b\n    from (select * from alltypesorc) v1)\nselect cdouble, a, b, a + b, cdouble + a from v2\norder by 1, 2, 3 limit 5","edges":[{"sources":[5],"targets":[0],"edgeType":"PROJECTION"},{"sources":[6,7,8,9,10,5,11,12,13,14,15,16],"targets":[1],"expression":"(tok_function count (. (tok_table_or_col alltypesorc) cint) (tok_windowspec (tok_partitioningspec (tok_distributeby 0) (tok_orderby (tok_tabsortcolnameasc 0))) (tok_windowrange (preceding 2147483647) (following 2147483647))))","edgeType":"PROJECTION"},{"sources":[6,7,8,9,10,5,11,12,13,14,15,16],"targets":[2],"expression":"(tok_function sum (+ (tok_function tok_bigint (. (tok_table_or_col alltypesorc) cint)) (. (tok_table_or_col alltypesorc) cbigint)) (tok_windowspec (tok_partitioningspec (tok_distributeby (. (tok_table_or_col alltypesorc) cbo
 olean1)) (tok_orderby (tok_tabsortcolnameasc (. (tok_table_or_col alltypesorc) cboolean1)))) (tok_windowrange (preceding 2147483647) (following 2147483647))))","edgeType":"PROJECTION"},{"sources":[6,7,8,9,10,5,11,12,13,14,15,16],"targets":[3],"expression":"((tok_function count (. (tok_table_or_col alltypesorc) cint) (tok_windowspec (tok_partitioningspec (tok_distributeby 0) (tok_orderby (tok_tabsortcolnameasc 0))) (tok_windowrange (preceding 2147483647) (following 2147483647)))) + (tok_function sum (+ (tok_function tok_bigint (. (tok_table_or_col alltypesorc) cint)) (. (tok_table_or_col alltypesorc) cbigint)) (tok_windowspec (tok_partitioningspec (tok_distributeby (. (tok_table_or_col alltypesorc) cboolean1)) (tok_orderby (tok_tabsortcolnameasc (. (tok_table_or_col alltypesorc) cboolean1)))) (tok_windowrange (preceding 2147483647) (following 2147483647)))))","edgeType":"PROJECTION"},{"sources":[5,6,7,8,9,10,11,12,13,14,15,16],"targets":[4],"expression":"(alltypesorc._col6 + UDFToDou
 ble((tok_function count (. (tok_table_or_col alltypesorc) cint) (tok_windowspec (tok_partitioningspec (tok_distributeby 0) (tok_orderby (tok_tabsortcolnameasc 0))) (tok_windowrange (preceding 2147483647) (following 2147483647))))))","edgeType":"PROJECTION"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"cdouble"},{"id":1,"vertexType":"COLUMN","vertexId":"a"},{"id":2,"vertexType":"COLUMN","vertexId":"b"},{"id":3,"vertexType":"COLUMN","vertexId":"c3"},{"id":4,"vertexType":"COLUMN","vertexId":"c4"},{"id":5,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cdouble"},{"id":6,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"},{"id":7,"vertexType":"COLUMN","vertexId":"default.alltypesorc.csmallint"},{"id":8,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"},{"id":9,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cbigint"},{"id":10,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cfloat"},{"id":11,"vertexType":"COLUMN","vertexId":"default.allty
 pesorc.cstring1"},{"id":12,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cstring2"},{"id":13,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctimestamp1"},{"id":14,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctimestamp2"},{"id":15,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean1"},{"id":16,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean2"}]}
 12205.0	9173	-919551973060	-919551963887	21378.0
 6723.0	9173	-919551973060	-919551963887	15896.0
 13776.0	9173	-919551973060	-919551963887	22949.0
@@ -651,128 +87,7 @@ order by 1, 2, 3, 4 limit 5
 PREHOOK: type: QUERY
 PREHOOK: Input: default@alltypesorc
 #### A masked pattern was here ####
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "890b55d7a07682c67354eb0cabb1045e",
-  "queryText": "select a.cbigint, a.ctinyint, b.cint, b.ctinyint\nfrom\n  (select ctinyint, cbigint from alltypesorc\n   union all\n   select ctinyint, cbigint from alltypesorc) a\n  inner join\n  alltypesorc b\n  on (a.ctinyint = b.ctinyint)\nwhere b.ctinyint < 100\norder by 1, 2, 3, 4 limit 5",
-  "edges": [
-    {
-      "sources": [
-        4
-      ],
-      "targets": [
-        0
-      ],
-      "expression": "a.ctinyint",
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        5
-      ],
-      "targets": [
-        1
-      ],
-      "expression": "a.cbigint",
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        6
-      ],
-      "targets": [
-        2
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        5
-      ],
-      "targets": [
-        3
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        5
-      ],
-      "targets": [
-        0,
-        1,
-        2,
-        3
-      ],
-      "expression": "(alltypesorc.ctinyint < 100)",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [
-        5
-      ],
-      "targets": [
-        0,
-        1,
-        2,
-        3
-      ],
-      "expression": "(ctinyint < 100)",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [
-        5
-      ],
-      "targets": [
-        0,
-        1,
-        2,
-        3
-      ],
-      "expression": "(a.cbigint = alltypesorc.ctinyint)",
-      "edgeType": "PREDICATE"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "a.cbigint"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "a.ctinyint"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "b.cint"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "b.ctinyint"
-    },
-    {
-      "id": 4,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cbigint"
-    },
-    {
-      "id": 5,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.ctinyint"
-    },
-    {
-      "id": 6,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cint"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"890b55d7a07682c67354eb0cabb1045e","queryText":"select a.cbigint, a.ctinyint, b.cint, b.ctinyint\nfrom\n  (select ctinyint, cbigint from alltypesorc\n   union all\n   select ctinyint, cbigint from alltypesorc) a\n  inner join\n  alltypesorc b\n  on (a.ctinyint = b.ctinyint)\nwhere b.ctinyint < 100\norder by 1, 2, 3, 4 limit 5","edges":[{"sources":[4],"targets":[0],"expression":"a.ctinyint","edgeType":"PROJECTION"},{"sources":[5],"targets":[1],"expression":"a.cbigint","edgeType":"PROJECTION"},{"sources":[6],"targets":[2],"edgeType":"PROJECTION"},{"sources":[5],"targets":[3],"edgeType":"PROJECTION"},{"sources":[5],"targets":[0,1,2,3],"expression":"(alltypesorc.ctinyint < 100)","edgeType":"PREDICATE"},{"sources":[5],"targets":[0,1,2,3],"expression":"(ctinyint < 100)","edgeType":"PREDICATE"},{"sources":[5],"targets":[0,1,2,3],"expression":"(a.cbigint = alltypesorc.ctinyint)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexI
 d":"a.cbigint"},{"id":1,"vertexType":"COLUMN","vertexId":"a.ctinyint"},{"id":2,"vertexType":"COLUMN","vertexId":"b.cint"},{"id":3,"vertexType":"COLUMN","vertexId":"b.ctinyint"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cbigint"},{"id":5,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"},{"id":6,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"}]}
 -47746898	-51	440161865	-51
 -47746898	-51	NULL	-51
 -47746898	-51	-637509859	-51
@@ -791,197 +106,7 @@ and x.ctinyint + length(c.cstring2) < 1000
 PREHOOK: type: QUERY
 PREHOOK: Input: default@alltypesorc
 #### A masked pattern was here ####
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "3a12ad24b2622a8958df12d0bdc60f8a",
-  "queryText": "select x.ctinyint, x.cint, c.cbigint-100, c.cstring1\nfrom alltypesorc c\njoin (\n   select a.ctinyint ctinyint, b.cint cint\n   from (select * from alltypesorc a where cboolean1=false) a\n   join alltypesorc b on (a.cint = b.cbigint - 224870380)\n ) x on (x.cint = c.cint)\nwhere x.ctinyint > 10\nand x.cint < 4.5\nand x.ctinyint + length(c.cstring2) < 1000",
-  "edges": [
-    {
-      "sources": [
-        4
-      ],
-      "targets": [
-        0
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        5
-      ],
-      "targets": [
-        1
-      ],
-      "expression": "cint",
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        6
-      ],
-      "targets": [
-        2
-      ],
-      "expression": "(c.cbigint - UDFToLong(100))",
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        7
-      ],
-      "targets": [
-        3
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        5
-      ],
-      "targets": [
-        0,
-        1,
-        2,
-        3
-      ],
-      "expression": "(UDFToDouble(c.cint) < 4.5)",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [
-        8
-      ],
-      "targets": [
-        0,
-        1,
-        2,
-        3
-      ],
-      "expression": "((UDFToInteger(ctinyint) + length(c.cstring2)) < 1000)",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [
-        5
-      ],
-      "targets": [
-        0,
-        1,
-        2,
-        3
-      ],
-      "expression": "(c.cint = c.cint)",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [
-        9
-      ],
-      "targets": [
-        0,
-        1,
-        2,
-        3
-      ],
-      "expression": "(c.cboolean1 = false)",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [
-        4
-      ],
-      "targets": [
-        0,
-        1,
-        2,
-        3
-      ],
-      "expression": "(c.ctinyint > 10)",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [
-        6,
-        5
-      ],
-      "targets": [
-        0,
-        1,
-        2,
-        3
-      ],
-      "expression": "((c.cbigint - UDFToLong(224870380)) = UDFToLong(c.cint))",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [
-        4,
-        8
-      ],
-      "targets": [
-        0,
-        1,
-        2,
-        3
-      ],
-      "expression": "((UDFToInteger(c.ctinyint) + length(c.cstring2)) < 1000)",
-      "edgeType": "PREDICATE"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "x.ctinyint"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "x.cint"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "c2"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "c.cstring1"
-    },
-    {
-      "id": 4,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.ctinyint"
-    },
-    {
-      "id": 5,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cint"
-    },
-    {
-      "id": 6,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cbigint"
-    },
-    {
-      "id": 7,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cstring1"
-    },
-    {
-      "id": 8,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cstring2"
-    },
-    {
-      "id": 9,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cboolean1"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"3a12ad24b2622a8958df12d0bdc60f8a","queryText":"select x.ctinyint, x.cint, c.cbigint-100, c.cstring1\nfrom alltypesorc c\njoin (\n   select a.ctinyint ctinyint, b.cint cint\n   from (select * from alltypesorc a where cboolean1=false) a\n   join alltypesorc b on (a.cint = b.cbigint - 224870380)\n ) x on (x.cint = c.cint)\nwhere x.ctinyint > 10\nand x.cint < 4.5\nand x.ctinyint + length(c.cstring2) < 1000","edges":[{"sources":[4],"targets":[0],"edgeType":"PROJECTION"},{"sources":[5],"targets":[1],"expression":"cint","edgeType":"PROJECTION"},{"sources":[6],"targets":[2],"expression":"(c.cbigint - UDFToLong(100))","edgeType":"PROJECTION"},{"sources":[7],"targets":[3],"edgeType":"PROJECTION"},{"sources":[5],"targets":[0,1,2,3],"expression":"(UDFToDouble(c.cint) < 4.5)","edgeType":"PREDICATE"},{"sources":[8],"targets":[0,1,2,3],"expression":"((UDFToInteger(ctinyint) + length(c.cstring2)) < 1000)","edgeType":"PREDICATE"},{"sources":[5],"targets":[0,1,2
 ,3],"expression":"(c.cint = c.cint)","edgeType":"PREDICATE"},{"sources":[9],"targets":[0,1,2,3],"expression":"(c.cboolean1 = false)","edgeType":"PREDICATE"},{"sources":[4],"targets":[0,1,2,3],"expression":"(c.ctinyint > 10)","edgeType":"PREDICATE"},{"sources":[6,5],"targets":[0,1,2,3],"expression":"((c.cbigint - UDFToLong(224870380)) = UDFToLong(c.cint))","edgeType":"PREDICATE"},{"sources":[4,8],"targets":[0,1,2,3],"expression":"((UDFToInteger(c.ctinyint) + length(c.cstring2)) < 1000)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"x.ctinyint"},{"id":1,"vertexType":"COLUMN","vertexId":"x.cint"},{"id":2,"vertexType":"COLUMN","vertexId":"c2"},{"id":3,"vertexType":"COLUMN","vertexId":"c.cstring1"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"},{"id":5,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"},{"id":6,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cbigint"},{"id":7,"vertexType":"COLUMN","vertexId":"defau
 lt.alltypesorc.cstring1"},{"id":8,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cstring2"},{"id":9,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean1"}]}
 11	-654374827	857266369	OEfPnHnIYueoup
 PREHOOK: query: select c1, x2, x3
 from (
@@ -1004,99 +129,7 @@ order by x2, c1 desc
 PREHOOK: type: QUERY
 PREHOOK: Input: default@alltypesorc
 #### A masked pattern was here ####
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "bc64f8bec21631969a17930ec609cde9",
-  "queryText": "select c1, x2, x3\nfrom (\n  select c1, min(c2) x2, sum(c3) x3\n  from (\n    select c1, c2, c3\n    from (\n      select cint c1, ctinyint c2, min(cbigint) c3\n      from alltypesorc\n      where cint is not null\n      group by cint, ctinyint\n      order by cint, ctinyint\n      limit 5\n    ) x\n  ) x2\n  group by c1\n) y\nwhere x2 > 0\norder by x2, c1 desc",
-  "edges": [
-    {
-      "sources": [
-        3
-      ],
-      "targets": [
-        0
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        4
-      ],
-      "targets": [
-        1
-      ],
-      "expression": "min(default.alltypesorc.ctinyint)",
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        5
-      ],
-      "targets": [
-        2
-      ],
-      "expression": "sum(min(default.alltypesorc.cbigint))",
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        3
-      ],
-      "targets": [
-        0,
-        1,
-        2
-      ],
-      "expression": "alltypesorc.cint is not null",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [
-        4
-      ],
-      "targets": [
-        0,
-        1,
-        2
-      ],
-      "expression": "(min(default.alltypesorc.ctinyint) > 0)",
-      "edgeType": "PREDICATE"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "c1"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "x2"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "x3"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cint"
-    },
-    {
-      "id": 4,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.ctinyint"
-    },
-    {
-      "id": 5,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cbigint"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"bc64f8bec21631969a17930ec609cde9","queryText":"select c1, x2, x3\nfrom (\n  select c1, min(c2) x2, sum(c3) x3\n  from (\n    select c1, c2, c3\n    from (\n      select cint c1, ctinyint c2, min(cbigint) c3\n      from alltypesorc\n      where cint is not null\n      group by cint, ctinyint\n      order by cint, ctinyint\n      limit 5\n    ) x\n  ) x2\n  group by c1\n) y\nwhere x2 > 0\norder by x2, c1 desc","edges":[{"sources":[3],"targets":[0],"edgeType":"PROJECTION"},{"sources":[4],"targets":[1],"expression":"min(default.alltypesorc.ctinyint)","edgeType":"PROJECTION"},{"sources":[5],"targets":[2],"expression":"sum(min(default.alltypesorc.cbigint))","edgeType":"PROJECTION"},{"sources":[3],"targets":[0,1,2],"expression":"alltypesorc.cint is not null","edgeType":"PREDICATE"},{"sources":[4],"targets":[0,1,2],"expression":"(min(default.alltypesorc.ctinyint) > 0)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"c1"},
 {"id":1,"vertexType":"COLUMN","vertexId":"x2"},{"id":2,"vertexType":"COLUMN","vertexId":"x3"},{"id":3,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"},{"id":5,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cbigint"}]}
 -1072910839	11	2048385991
 -1073279343	11	-1595604468
 PREHOOK: query: select key, value from src1
@@ -1104,74 +137,7 @@ where key in (select key+18 from src1) order by key
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src1
 #### A masked pattern was here ####
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "8b9d63653e36ecf4dd425d3cc3de9199",
-  "queryText": "select key, value from src1\nwhere key in (select key+18 from src1) order by key",
-  "edges": [
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        3
-      ],
-      "targets": [
-        1
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [],
-      "targets": [
-        0,
-        1
-      ],
-      "expression": "(1 = 1)",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0,
-        1
-      ],
-      "expression": "(UDFToDouble(src1.key) = (UDFToDouble(src1.key) + UDFToDouble(18)))",
-      "edgeType": "PREDICATE"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "key"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "value"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.key"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.value"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"8b9d63653e36ecf4dd425d3cc3de9199","queryText":"select key, value from src1\nwhere key in (select key+18 from src1) order by key","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"},{"sources":[],"targets":[0,1],"expression":"(1 = 1)","edgeType":"PREDICATE"},{"sources":[2],"targets":[0,1],"expression":"(UDFToDouble(src1.key) = (UDFToDouble(src1.key) + UDFToDouble(18)))","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"key"},{"id":1,"vertexType":"COLUMN","vertexId":"value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"default.src1.value"}]}
 146	val_146
 273	val_273
 PREHOOK: query: select * from src1 a
@@ -1183,91 +149,7 @@ PREHOOK: type: QUERY
 PREHOOK: Input: default@alltypesorc
 PREHOOK: Input: default@src1
 #### A masked pattern was here ####
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "8bf193b0658183be94e2428a79d91d10",
-  "queryText": "select * from src1 a\nwhere exists\n  (select cint from alltypesorc b\n   where a.key = b.ctinyint + 300)\nand key > 300",
-  "edges": [
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        3
-      ],
-      "targets": [
-        1
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0,
-        1
-      ],
-      "expression": "(UDFToDouble(a.key) > UDFToDouble(300))",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [],
-      "targets": [
-        0,
-        1
-      ],
-      "expression": "(1 = 1)",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [
-        2,
-        4
-      ],
-      "targets": [
-        0,
-        1
-      ],
-      "expression": "(UDFToDouble(a.key) = UDFToDouble((UDFToInteger(b.ctinyint) + 300)))",
-      "edgeType": "PREDICATE"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "a.key"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "a.value"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.key"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.value"
-    },
-    {
-      "id": 4,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.ctinyint"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"8bf193b0658183be94e2428a79d91d10","queryText":"select * from src1 a\nwhere exists\n  (select cint from alltypesorc b\n   where a.key = b.ctinyint + 300)\nand key > 300","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"},{"sources":[2],"targets":[0,1],"expression":"(UDFToDouble(a.key) > UDFToDouble(300))","edgeType":"PREDICATE"},{"sources":[],"targets":[0,1],"expression":"(1 = 1)","edgeType":"PREDICATE"},{"sources":[2,4],"targets":[0,1],"expression":"(UDFToDouble(a.key) = UDFToDouble((UDFToInteger(b.ctinyint) + 300)))","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"a.key"},{"id":1,"vertexType":"COLUMN","vertexId":"a.value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"default.src1.value"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"}]}
 311	val_311
 Warning: Shuffle Join JOIN[18][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
 PREHOOK: query: select key, value from src1
@@ -1275,121 +157,7 @@ where key not in (select key+18 from src1) order by key
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src1
 #### A masked pattern was here ####
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "9b488fe1d7cf018aad3825173808cd36",
-  "queryText": "select key, value from src1\nwhere key not in (select key+18 from src1) order by key",
-  "edges": [
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0
-      ],
-      "expression": "key",
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        3
-      ],
-      "targets": [
-        1
-      ],
-      "expression": "value",
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [],
-      "targets": [
-        0,
-        1
-      ],
-      "expression": "(1 = 1)",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [],
-      "targets": [
-        0,
-        1
-      ],
-      "expression": "_o__c0 is null",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0,
-        1
-      ],
-      "expression": "(UDFToDouble(src1.key) + UDFToDouble(18)) is null",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [
-        4
-      ],
-      "targets": [
-        0,
-        1
-      ],
-      "expression": "(count(*) = 0)",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [],
-      "targets": [
-        0,
-        1
-      ],
-      "expression": "true",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0,
-        1
-      ],
-      "expression": "(UDFToDouble(key) = (UDFToDouble(src1.key) + UDFToDouble(18)))",
-      "edgeType": "PREDICATE"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "key"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "value"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.key"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.value"
-    },
-    {
-      "id": 4,
-      "vertexType": "TABLE",
-      "vertexId": "default.src1"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"9b488fe1d7cf018aad3825173808cd36","queryText":"select key, value from src1\nwhere key not in (select key+18 from src1) order by key","edges":[{"sources":[2],"targets":[0],"expression":"key","edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"expression":"value","edgeType":"PROJECTION"},{"sources":[],"targets":[0,1],"expression":"(1 = 1)","edgeType":"PREDICATE"},{"sources":[],"targets":[0,1],"expression":"_o__c0 is null","edgeType":"PREDICATE"},{"sources":[2],"targets":[0,1],"expression":"(UDFToDouble(src1.key) + UDFToDouble(18)) is null","edgeType":"PREDICATE"},{"sources":[4],"targets":[0,1],"expression":"(count(*) = 0)","edgeType":"PREDICATE"},{"sources":[],"targets":[0,1],"expression":"true","edgeType":"PREDICATE"},{"sources":[2],"targets":[0,1],"expression":"(UDFToDouble(key) = (UDFToDouble(src1.key) + UDFToDouble(18)))","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"key"},{"id":1,"vertexType":"COLUMN","ver
 texId":"value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"default.src1.value"},{"id":4,"vertexType":"TABLE","vertexId":"default.src1"}]}
 PREHOOK: query: select * from src1 a
 where not exists
   (select cint from alltypesorc b
@@ -1399,111 +167,7 @@ PREHOOK: type: QUERY
 PREHOOK: Input: default@alltypesorc
 PREHOOK: Input: default@src1
 #### A masked pattern was here ####
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "53191056e05af9080a30de853e8cea9c",
-  "queryText": "select * from src1 a\nwhere not exists\n  (select cint from alltypesorc b\n   where a.key = b.ctinyint + 300)\nand key > 300",
-  "edges": [
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        3
-      ],
-      "targets": [
-        1
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0,
-        1
-      ],
-      "expression": "(UDFToDouble(a.key) > UDFToDouble(300))",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [],
-      "targets": [
-        0,
-        1
-      ],
-      "expression": "sq_corr_0 is null",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [],
-      "targets": [
-        0,
-        1
-      ],
-      "expression": "(1 = 1)",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [
-        2,
-        4
-      ],
-      "targets": [
-        0,
-        1
-      ],
-      "expression": "(UDFToDouble(a.key) = UDFToDouble((UDFToInteger(b.ctinyint) + 300)))",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [
-        4
-      ],
-      "targets": [
-        0,
-        1
-      ],
-      "expression": "(UDFToInteger(b.ctinyint) + 300) is null",
-      "edgeType": "PREDICATE"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "a.key"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "a.value"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.key"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.value"
-    },
-    {
-      "id": 4,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.ctinyint"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"53191056e05af9080a30de853e8cea9c","queryText":"select * from src1 a\nwhere not exists\n  (select cint from alltypesorc b\n   where a.key = b.ctinyint + 300)\nand key > 300","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"},{"sources":[2],"targets":[0,1],"expression":"(UDFToDouble(a.key) > UDFToDouble(300))","edgeType":"PREDICATE"},{"sources":[],"targets":[0,1],"expression":"sq_corr_0 is null","edgeType":"PREDICATE"},{"sources":[],"targets":[0,1],"expression":"(1 = 1)","edgeType":"PREDICATE"},{"sources":[2,4],"targets":[0,1],"expression":"(UDFToDouble(a.key) = UDFToDouble((UDFToInteger(b.ctinyint) + 300)))","edgeType":"PREDICATE"},{"sources":[4],"targets":[0,1],"expression":"(UDFToInteger(b.ctinyint) + 300) is null","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"a.key"},{"id":1,"vertexType":"COLUMN","vertexId":"a.value"},{"id":2,"vertexType":"COLUM
 N","vertexId":"default.src1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"default.src1.value"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"}]}
 369	
 401	val_401
 406	val_406
@@ -1512,76 +176,7 @@ select x, y from t where y > 'v' order by x, y limit 5
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src1
 #### A masked pattern was here ####
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "d40d80b93db06c12df9a6ccdc108a9d1",
-  "queryText": "with t as (select key x, value y from src1 where key > '2')\nselect x, y from t where y > 'v' order by x, y limit 5",
-  "edges": [
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        3
-      ],
-      "targets": [
-        1
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0,
-        1
-      ],
-      "expression": "(src1.key > '2')",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [
-        3
-      ],
-      "targets": [
-        0,
-        1
-      ],
-      "expression": "(src1.value > 'v')",
-      "edgeType": "PREDICATE"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "x"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "y"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.key"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.value"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"d40d80b93db06c12df9a6ccdc108a9d1","queryText":"with t as (select key x, value y from src1 where key > '2')\nselect x, y from t where y > 'v' order by x, y limit 5","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"},{"sources":[2],"targets":[0,1],"expression":"(src1.key > '2')","edgeType":"PREDICATE"},{"sources":[3],"targets":[0,1],"expression":"(src1.value > 'v')","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"x"},{"id":1,"vertexType":"COLUMN","vertexId":"y"},{"id":2,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"default.src1.value"}]}
 213	val_213
 238	val_238
 255	val_255
@@ -1592,76 +187,7 @@ select x, y where y > 'v' order by x, y limit 5
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src1
 #### A masked pattern was here ####
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "9180b71a610dbcf5e636a3c03e48ca3b",
-  "queryText": "from (select key x, value y from src1 where key > '2') t\nselect x, y where y > 'v' order by x, y limit 5",
-  "edges": [
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        3
-      ],
-      "targets": [
-        1
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0,
-        1
-      ],
-      "expression": "(src1.key > '2')",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [
-        3
-      ],
-      "targets": [
-        0,
-        1
-      ],
-      "expression": "(src1.value > 'v')",
-      "edgeType": "PREDICATE"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "x"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "y"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.key"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.value"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"9180b71a610dbcf5e636a3c03e48ca3b","queryText":"from (select key x, value y from src1 where key > '2') t\nselect x, y where y > 'v' order by x, y limit 5","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"},{"sources":[2],"targets":[0,1],"expression":"(src1.key > '2')","edgeType":"PREDICATE"},{"sources":[3],"targets":[0,1],"expression":"(src1.value > 'v')","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"x"},{"id":1,"vertexType":"COLUMN","vertexId":"y"},{"id":2,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"default.src1.value"}]}
 213	val_213
 238	val_238
 255	val_255
@@ -1675,129 +201,13 @@ PREHOOK: type: CREATEVIEW
 PREHOOK: Input: default@alltypesorc
 PREHOOK: Output: database:default
 PREHOOK: Output: default@dest_v1
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "a3b2d2665c90fd669400f247f751f081",
-  "queryText": "create view dest_v1 as\n  select ctinyint, cint from alltypesorc where ctinyint is not null",
-  "edges": [
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        3
-      ],
-      "targets": [
-        1
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0,
-        1
-      ],
-      "expression": "alltypesorc.ctinyint is not null",
-      "edgeType": "PREDICATE"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_v1.ctinyint"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_v1.cint"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.ctinyint"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cint"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"a3b2d2665c90fd669400f247f751f081","queryText":"create view dest_v1 as\n  select ctinyint, cint from alltypesorc where ctinyint is not null","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"},{"sources":[2],"targets":[0,1],"expression":"alltypesorc.ctinyint is not null","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.dest_v1.ctinyint"},{"id":1,"vertexType":"COLUMN","vertexId":"default.dest_v1.cint"},{"id":2,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"},{"id":3,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"}]}
 PREHOOK: query: select * from dest_v1 order by ctinyint, cint limit 2
 PREHOOK: type: QUERY
 PREHOOK: Input: default@alltypesorc
 PREHOOK: Input: default@dest_v1
 #### A masked pattern was here ####
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "75e07b246069a5541af4a3983500b439",
-  "queryText": "select * from dest_v1 order by ctinyint, cint limit 2",
-  "edges": [
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        3
-      ],
-      "targets": [
-        1
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0,
-        1
-      ],
-      "expression": "alltypesorc.ctinyint is not null",
-      "edgeType": "PREDICATE"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "dest_v1.ctinyint"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "dest_v1.cint"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.ctinyint"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cint"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"75e07b246069a5541af4a3983500b439","queryText":"select * from dest_v1 order by ctinyint, cint limit 2","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"},{"sources":[2],"targets":[0,1],"expression":"alltypesorc.ctinyint is not null","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"dest_v1.ctinyint"},{"id":1,"vertexType":"COLUMN","vertexId":"dest_v1.cint"},{"id":2,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"},{"id":3,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"}]}
 -64	NULL
 -64	NULL
 PREHOOK: query: alter view dest_v1 as select ctinyint from alltypesorc
@@ -1805,90 +215,14 @@ PREHOOK: type: CREATEVIEW
 PREHOOK: Input: default@alltypesorc
 PREHOOK: Output: database:default
 PREHOOK: Output: default@dest_v1
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "bcab8b0c498b0d94e0967170956392b6",
-  "queryText": "alter view dest_v1 as select ctinyint from alltypesorc",
-  "edges": [
-    {
-      "sources": [
-        1
-      ],
-      "targets": [
-        0
-      ],
-      "edgeType": "PROJECTION"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_v1.ctinyint"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.ctinyint"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"bcab8b0c498b0d94e0967170956392b6","queryText":"alter view dest_v1 as select ctinyint from alltypesorc","edges":[{"sources":[1],"targets":[0],"edgeType":"PROJECTION"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.dest_v1.ctinyint"},{"id":1,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"}]}
 PREHOOK: query: select t.ctinyint from (select * from dest_v1 where ctinyint is not null) t
 where ctinyint > 10 order by ctinyint limit 2
 PREHOOK: type: QUERY
 PREHOOK: Input: default@alltypesorc
 PREHOOK: Input: default@dest_v1
 #### A masked pattern was here ####
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "b0192d4da86f4bef38fe7ab1fc607906",
-  "queryText": "select t.ctinyint from (select * from dest_v1 where ctinyint is not null) t\nwhere ctinyint > 10 order by ctinyint limit 2",
-  "edges": [
-    {
-      "sources": [
-        1
-      ],
-      "targets": [
-        0
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        1
-      ],
-      "targets": [
-        0
-      ],
-      "expression": "alltypesorc.ctinyint is not null",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [
-        1
-      ],
-      "targets": [
-        0
-      ],
-      "expression": "(alltypesorc.ctinyint > 10)",
-      "edgeType": "PREDICATE"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "t.ctinyint"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.ctinyint"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"b0192d4da86f4bef38fe7ab1fc607906","queryText":"select t.ctinyint from (select * from dest_v1 where ctinyint is not null) t\nwhere ctinyint > 10 order by ctinyint limit 2","edges":[{"sources":[1],"targets":[0],"edgeType":"PROJECTION"},{"sources":[1],"targets":[0],"expression":"alltypesorc.ctinyint is not null","edgeType":"PREDICATE"},{"sources":[1],"targets":[0],"expression":"(alltypesorc.ctinyint > 10)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"t.ctinyint"},{"id":1,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"}]}
 11
 11
 PREHOOK: query: drop view if exists dest_v2
@@ -1913,55 +247,7 @@ PREHOOK: type: CREATEVIEW
 PREHOOK: Input: default@alltypesorc
 PREHOOK: Output: database:default
 PREHOOK: Output: default@dest_v2
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "eda442b42b9c3a9cbdb7aff1984ad2dd",
-  "queryText": "create view dest_v2 (a, b) as select c1, x2\nfrom (\n  select c1, min(c2) x2\n  from (\n    select c1, c2, c3\n    from (\n      select cint c1, ctinyint c2, min(cfloat) c3\n      from alltypesorc\n      group by cint, ctinyint\n      order by cint, ctinyint\n      limit 1\n    ) x\n  ) x2\n  group by c1\n) y\norder by x2,c1 desc",
-  "edges": [
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        3
-      ],
-      "targets": [
-        1
-      ],
-      "expression": "min(default.alltypesorc.ctinyint)",
-      "edgeType": "PROJECTION"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_v2.c1"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_v2.x2"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cint"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.ctinyint"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"eda442b42b9c3a9cbdb7aff1984ad2dd","queryText":"create view dest_v2 (a, b) as select c1, x2\nfrom (\n  select c1, min(c2) x2\n  from (\n    select c1, c2, c3\n    from (\n      select cint c1, ctinyint c2, min(cfloat) c3\n      from alltypesorc\n      group by cint, ctinyint\n      order by cint, ctinyint\n      limit 1\n    ) x\n  ) x2\n  group by c1\n) y\norder by x2,c1 desc","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"expression":"min(default.alltypesorc.ctinyint)","edgeType":"PROJECTION"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.dest_v2.c1"},{"id":1,"vertexType":"COLUMN","vertexId":"default.dest_v2.x2"},{"id":2,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"},{"id":3,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"}]}
 PREHOOK: query: drop view if exists dest_v3
 PREHOOK: type: DROPVIEW
 PREHOOK: query: create view dest_v3 (a1, a2, a3, a4, a5, a6, a7) as
@@ -1982,213 +268,7 @@ PREHOOK: type: CREATEVIEW
 PREHOOK: Input: default@alltypesorc
 PREHOOK: Output: database:default
 PREHOOK: Output: default@dest_v3
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "a0c2481ce1c24895a43a950f93a10da7",
-  "queryText": "create view dest_v3 (a1, a2, a3, a4, a5, a6, a7) as\n  select x.csmallint, x.cbigint bint1, x.ctinyint, c.cbigint bint2, x.cint, x.cfloat, c.cstring1\n  from alltypesorc c\n  join (\n     select a.csmallint csmallint, a.ctinyint ctinyint, a.cstring2 cstring2,\n           a.cint cint, a.cstring1 ctring1, b.cfloat cfloat, b.cbigint cbigint\n     from ( select * from alltypesorc a where cboolean1=true ) a\n     join alltypesorc b on (a.csmallint = b.cint)\n   ) x on (x.ctinyint = c.cbigint)\n  where x.csmallint=11\n  and x.cint > 899\n  and x.cfloat > 4.5\n  and c.cstring1 < '7'\n  and x.cint + x.cfloat + length(c.cstring1) < 1000",
-  "edges": [
-    {
-      "sources": [
-        7
-      ],
-      "targets": [
-        0
-      ],
-      "expression": "x._col15",
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        8
-      ],
-      "targets": [
-        1,
-        2
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        9
-      ],
-      "targets": [
-        3
-      ],
-      "expression": "x._col16",
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        10
-      ],
-      "targets": [
-        4
-      ],
-      "expression": "x._col18",
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        11
-      ],
-      "targets": [
-        5
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        12
-      ],
-      "targets": [
-        6
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        13
-      ],
-      "targets": [
-        0,
-        1,
-        3,
-        2,
-        4,
-        5,
-        6
-      ],
-      "expression": "(a.cboolean1 = true)",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [
-        7,
-        10,
-        12,
-        11
-      ],
-      "targets": [
-        0,
-        1,
-        3,
-        2,
-        4,
-        5,
-        6
-      ],
-      "expression": "(((((x.csmallint = 11) and (x.cint > 899)) and (x.cfloat > 4.5)) and (c.cstring1 < '7')) and (((x.cint + x.cfloat) + length(c.cstring1)) < 1000))",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [
-        7,
-        10
-      ],
-      "targets": [
-        0,
-        1,
-        3,
-        2,
-        4,
-        5,
-        6
-      ],
-      "expression": "(UDFToInteger(a._col1) = b.cint)",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [
-        8,
-        9
-      ],
-      "targets": [
-        0,
-        1,
-        3,
-        2,
-        4,
-        5,
-        6
-      ],
-      "expression": "(c.cbigint = UDFToLong(x._col1))",
-      "edgeType": "PREDICATE"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_v3.csmallint"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_v3.bint1"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_v3.bint2"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_v3.ctinyint"
-    },
-    {
-      "id": 4,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_v3.cint"
-    },
-    {
-      "id": 5,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_v3.cfloat"
-    },
-    {
-      "id": 6,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_v3.cstring1"
-    },
-    {
-      "id": 7,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.csmallint"
-    },
-    {
-      "id": 8,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cbigint"
-    },
-    {
-      "id": 9,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.ctinyint"
-    },
-    {
-      "id": 10,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cint"
-    },
-    {
-      "id": 11,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cfloat"
-    },
-    {
-      "id": 12,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cstring1"
-    },
-    {
-      "id": 13,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cboolean1"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"a0c2481ce1c24895a43a950f93a10da7","queryText":"create view dest_v3 (a1, a2, a3, a4, a5, a6, a7) as\n  select x.csmallint, x.cbigint bint1, x.ctinyint, c.cbigint bint2, x.cint, x.cfloat, c.cstring1\n  from alltypesorc c\n  join (\n     select a.csmallint csmallint, a.ctinyint ctinyint, a.cstring2 cstring2,\n           a.cint cint, a.cstring1 ctring1, b.cfloat cfloat, b.cbigint cbigint\n     from ( select * from alltypesorc a where cboolean1=true ) a\n     join alltypesorc b on (a.csmallint = b.cint)\n   ) x on (x.ctinyint = c.cbigint)\n  where x.csmallint=11\n  and x.cint > 899\n  and x.cfloat > 4.5\n  and c.cstring1 < '7'\n  and x.cint + x.cfloat + length(c.cstring1) < 1000","edges":[{"sources":[7],"targets":[0],"expression":"x._col15","edgeType":"PROJECTION"},{"sources":[8],"targets":[1,2],"edgeType":"PROJECTION"},{"sources":[9],"targets":[3],"expression":"x._col16","edgeType":"PROJECTION"},{"sources":[10],"targets":[4],"expression":"x._col18"
 ,"edgeType":"PROJECTION"},{"sources":[11],"targets":[5],"edgeType":"PROJECTION"},{"sources":[12],"targets":[6],"edgeType":"PROJECTION"},{"sources":[13],"targets":[0,1,3,2,4,5,6],"expression":"(a.cboolean1 = true)","edgeType":"PREDICATE"},{"sources":[7,10,12,11],"targets":[0,1,3,2,4,5,6],"expression":"(((((x.csmallint = 11) and (x.cint > 899)) and (x.cfloat > 4.5)) and (c.cstring1 < '7')) and (((x.cint + x.cfloat) + length(c.cstring1)) < 1000))","edgeType":"PREDICATE"},{"sources":[7,10],"targets":[0,1,3,2,4,5,6],"expression":"(UDFToInteger(a._col1) = b.cint)","edgeType":"PREDICATE"},{"sources":[8,9],"targets":[0,1,3,2,4,5,6],"expression":"(c.cbigint = UDFToLong(x._col1))","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.dest_v3.csmallint"},{"id":1,"vertexType":"COLUMN","vertexId":"default.dest_v3.bint1"},{"id":2,"vertexType":"COLUMN","vertexId":"default.dest_v3.bint2"},{"id":3,"vertexType":"COLUMN","vertexId":"default.dest_v3.ctinyint"},{"id":4,"
 vertexType":"COLUMN","vertexId":"default.dest_v3.cint"},{"id":5,"vertexType":"COLUMN","vertexId":"default.dest_v3.cfloat"},{"id":6,"vertexType":"COLUMN","vertexId":"default.dest_v3.cstring1"},{"id":7,"vertexType":"COLUMN","vertexId":"default.alltypesorc.csmallint"},{"id":8,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cbigint"},{"id":9,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"},{"id":10,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"},{"id":11,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cfloat"},{"id":12,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cstring1"},{"id":13,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean1"}]}
 PREHOOK: query: alter view dest_v3 as
   select * from (
     select sum(a.ctinyint) over (partition by a.csmallint order by a.csmallint) a,
@@ -2202,281 +282,12 @@ PREHOOK: type: CREATEVIEW
 PREHOOK: Input: default@alltypesorc
 PREHOOK: Output: database:default
 PREHOOK: Output: default@dest_v3
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "949093880975cc807ad1a8003e8a8c7c",
-  "queryText": "alter view dest_v3 as\n  select * from (\n    select sum(a.ctinyint) over (partition by a.csmallint order by a.csmallint) a,\n      count(b.cstring1) x, b.cboolean1\n    from alltypesorc a join alltypesorc b on (a.cint = b.cint)\n    where a.cboolean2 = true and b.cfloat > 0\n    group by a.ctinyint, a.csmallint, b.cboolean1\n    having count(a.cint) > 10\n    order by a, x, b.cboolean1 limit 10) t",
-  "edges": [
-    {
-      "sources": [
-        3,
-        4,
-        5,
-        6,
-        7
-      ],
-      "targets": [
-        0
-      ],
-      "expression": "(tok_function sum (. (tok_table_or_col a) ctinyint) (tok_windowspec (tok_partitioningspec (tok_distributeby (. (tok_table_or_col a) csmallint)) (tok_orderby (tok_tabsortcolnameasc (. (tok_table_or_col a) csmallint))))))",
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        6
-      ],
-      "targets": [
-        1
-      ],
-      "expression": "count(default.alltypesorc.cstring1)",
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        5
-      ],
-      "targets": [
-        2
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        8,
-        9
-      ],
-      "targets": [
-        0,
-        1,
-        2
-      ],
-      "expression": "((a.cboolean2 = true) and (b.cfloat > 0.0))",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [
-        7
-      ],
-      "targets": [
-        0,
-        1,
-        2
-      ],
-      "expression": "(count(default.alltypesorc.cint) > 10)",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [
-        7
-      ],
-      "targets": [
-        0,
-        1,
-        2
-      ],
-      "expression": "(a.cint = b.cint)",
-      "edgeType": "PREDICATE"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_v3.a"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_v3.x"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_v3.cboolean1"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.ctinyint"
-    },
-    {
-      "id": 4,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.csmallint"
-    },
-    {
-      "id": 5,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cboolean1"
-    },
-    {
-      "id": 6,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cstring1"
-    },
-    {
-      "id": 7,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cint"
-    },
-    {
-      "id": 8,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cboolean2"
-    },
-    {
-      "id": 9,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cfloat"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"949093880975cc807ad1a8003e8a8c7c","queryText":"alter view dest_v3 as\n  select * from (\n    select sum(a.ctinyint) over (partition by a.csmallint order by a.csmallint) a,\n      count(b.cstring1) x, b.cboolean1\n    from alltypesorc a join alltypesorc b on (a.cint = b.cint)\n    where a.cboolean2 = true and b.cfloat > 0\n    group by a.ctinyint, a.csmallint, b.cboolean1\n    having count(a.cint) > 10\n    order by a, x, b.cboolean1 limit 10) t","edges":[{"sources":[3,4,5,6,7],"targets":[0],"expression":"(tok_function sum (. (tok_table_or_col a) ctinyint) (tok_windowspec (tok_partitioningspec (tok_distributeby (. (tok_table_or_col a) csmallint)) (tok_orderby (tok_tabsortcolnameasc (. (tok_table_or_col a) csmallint))))))","edgeType":"PROJECTION"},{"sources":[6],"targets":[1],"expression":"count(default.alltypesorc.cstring1)","edgeType":"PROJECTION"},{"sources":[5],"targets":[2],"edgeType":"PROJECTION"},{"sources":[8,9],"targets":[0,1,2],"express
 ion":"((a.cboolean2 = true) and (b.cfloat > 0.0))","edgeType":"PREDICATE"},{"sources":[7],"targets":[0,1,2],"expression":"(count(default.alltypesorc.cint) > 10)","edgeType":"PREDICATE"},{"sources":[7],"targets":[0,1,2],"expression":"(a.cint = b.cint)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.dest_v3.a"},{"id":1,"vertexType":"COLUMN","vertexId":"default.dest_v3.x"},{"id":2,"vertexType":"COLUMN","vertexId":"default.dest_v3.cboolean1"},{"id":3,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.csmallint"},{"id":5,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean1"},{"id":6,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cstring1"},{"id":7,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"},{"id":8,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean2"},{"id":9,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cfloat"}]}
 PREHOOK: query: select * from dest_v3 limit 2
 PREHOOK: type: QUERY
 PREHOOK: Input: default@alltypesorc
 PREHOOK: Input: default@dest_v3
 #### A masked pattern was here ####
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "40bccc0722002f798d0548b59e369e83",
-  "queryText": "select * from dest_v3 limit 2",
-  "edges": [
-    {
-      "sources": [
-        3,
-        4,
-        5,
-        6,
-        7
-      ],
-      "targets": [
-        0
-      ],
-      "expression": "(tok_function sum (. (tok_table_or_col $hdt$_0) $f0) (tok_windowspec (tok_partitioningspec (tok_distributeby (. (tok_table_or_col $hdt$_0) $f1)) (tok_orderby (tok_tabsortcolnameasc (. (tok_table_or_col $hdt$_0) $f1)))) (tok_windowvalues (preceding 2147483647) current)))",
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        6
-      ],
-      "targets": [
-        1
-      ],
-      "expression": "count(default.alltypesorc.cstring1)",
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        5
-      ],
-      "targets": [
-        2
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        8
-      ],
-      "targets": [
-        0,
-        1,
-        2
-      ],
-      "expression": "(a.cboolean2 = true)",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [
-        7
-      ],
-      "targets": [
-        0,
-        1,
-        2
-      ],
-      "expression": "(count(default.alltypesorc.cint) > 10)",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [
-        9
-      ],
-      "targets": [
-        0,
-        1,
-        2
-      ],
-      "expression": "(a.cfloat > 0.0)",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [
-        7
-      ],
-      "targets": [
-        0,
-        1,
-        2
-      ],
-      "expression": "(a.cint = a.cint)",
-      "edgeType": "PREDICATE"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "dest_v3.a"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "dest_v3.x"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "dest_v3.cboolean1"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.ctinyint"
-    },
-    {
-      "id": 4,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.csmallint"
-    },
-    {
-      "id": 5,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cboolean1"
-    },
-    {
-      "id": 6,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cstring1"
-    },
-    {
-      "id": 7,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cint"
-    },
-    {
-      "id": 8,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cboolean2"
-    },
-    {
-      "id": 9,
-      "vertexType": "COLUMN",
-      "vertexId": "default.alltypesorc.cfloat"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"40bccc0722002f798d0548b59e369e83","queryText":"select * from dest_v3 limit 2","edges":[{"sources":[3,4,5,6,7],"targets":[0],"expression":"(tok_function sum (. (tok_table_or_col $hdt$_0) $f0) (tok_windowspec (tok_partitioningspec (tok_distributeby (. (tok_table_or_col $hdt$_0) $f1)) (tok_orderby (tok_tabsortcolnameasc (. (tok_table_or_col $hdt$_0) $f1)))) (tok_windowvalues (preceding 2147483647) current)))","edgeType":"PROJECTION"},{"sources":[6],"targets":[1],"expression":"count(default.alltypesorc.cstring1)","edgeType":"PROJECTION"},{"sources":[5],"targets":[2],"edgeType":"PROJECTION"},{"sources":[8],"targets":[0,1,2],"expression":"(a.cboolean2 = true)","edgeType":"PREDICATE"},{"sources":[7],"targets":[0,1,2],"expression":"(count(default.alltypesorc.cint) > 10)","edgeType":"PREDICATE"},{"sources":[9],"targets":[0,1,2],"expression":"(a.cfloat > 0.0)","edgeType":"PREDICATE"},{"sources":[7],"targets":[0,1,2],"expression":"(a.cint = a.cint)","edge
 Type":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"dest_v3.a"},{"id":1,"vertexType":"COLUMN","vertexId":"dest_v3.x"},{"id":2,"vertexType":"COLUMN","vertexId":"dest_v3.cboolean1"},{"id":3,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.csmallint"},{"id":5,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean1"},{"id":6,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cstring1"},{"id":7,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"},{"id":8,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean2"},{"id":9,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cfloat"}]}
 38	216	false
 38	229	true


[09/50] [abbrv] hive git commit: HIVE-11223: CBO (Calcite Return Path): MapJoin and SMBJoin conversion not triggered (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by ga...@apache.org.
HIVE-11223: CBO (Calcite Return Path): MapJoin and SMBJoin conversion not triggered (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/hbase-metastore
Commit: 4d984bded32b23fd1a6306dfdd6d9b458bdf400d
Parents: 8662d9d
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Fri Jul 10 02:01:17 2015 +0100
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Wed Jul 15 18:37:59 2015 +0100

----------------------------------------------------------------------
 .../calcite/translator/HiveOpConverter.java     |   4 +-
 .../translator/HiveOpConverterPostProc.java     |  34 +-
 .../queries/clientpositive/cbo_rp_auto_join0.q  |   1 +
 .../queries/clientpositive/cbo_rp_auto_join1.q  |   1 +
 .../test/queries/clientpositive/cbo_rp_join0.q  |   1 +
 .../clientpositive/cbo_rp_auto_join0.q.out      |  32 +-
 .../clientpositive/cbo_rp_auto_join1.q.out      | 578 ++++++++++---------
 .../results/clientpositive/cbo_rp_join0.q.out   | 185 +++---
 8 files changed, 471 insertions(+), 365 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/4d984bde/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 c711406..c54a601 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
@@ -211,7 +211,7 @@ public class HiveOpConverter {
     Map<Integer, ColumnInfo> posToNonPartColInfo = ht.getNonPartColInfoMap();
     List<Integer> neededColIndxsFrmReloptHT = scanRel.getNeededColIndxsFrmReloptHT();
     List<String> scanColNames = scanRel.getRowType().getFieldNames();
-    String tableAlias = scanRel.getTableAlias();
+    String tableAlias = scanRel.getConcatQbIDAlias();
 
     String colName;
     ColumnInfo colInfo;
@@ -251,7 +251,7 @@ public class HiveOpConverter {
     // 2. Setup TableScan
     TableScanOperator ts = (TableScanOperator) OperatorFactory.get(tsd, new RowSchema(colInfos));
 
-    topOps.put(scanRel.getConcatQbIDAlias(), ts);
+    topOps.put(tableAlias, ts);
 
     if (LOG.isDebugEnabled()) {
       LOG.debug("Generated " + ts + " with row schema: [" + ts.getSchema() + "]");

http://git-wip-us.apache.org/repos/asf/hive/blob/4d984bde/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverterPostProc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverterPostProc.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverterPostProc.java
index d861682..5080992 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverterPostProc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverterPostProc.java
@@ -51,7 +51,7 @@ public class HiveOpConverterPostProc implements Transform {
   private static final Log LOG = LogFactory.getLog(HiveOpConverterPostProc.class);
 
   private ParseContext                                  pctx;
-  private Map<String, Operator<? extends OperatorDesc>> aliasToJoinOpInfo;
+  private Map<String, Operator<? extends OperatorDesc>> aliasToOpInfo;
 
   @Override
   public ParseContext transform(ParseContext pctx) throws SemanticException {
@@ -66,11 +66,12 @@ public class HiveOpConverterPostProc implements Transform {
 
     // 1. Initialize aux data structures
     this.pctx = pctx;
-    this.aliasToJoinOpInfo = new HashMap<String, Operator<? extends OperatorDesc>>();
+    this.aliasToOpInfo = new HashMap<String, Operator<? extends OperatorDesc>>();
 
     // 2. Trigger transformation
     Map<Rule, NodeProcessor> opRules = new LinkedHashMap<Rule, NodeProcessor>();
     opRules.put(new RuleRegExp("R1", JoinOperator.getOperatorName() + "%"), new JoinAnnotate());
+    opRules.put(new RuleRegExp("R2", TableScanOperator.getOperatorName() + "%"), new TableScanAnnotate());
 
     Dispatcher disp = new DefaultRuleDispatcher(null, opRules, null);
     GraphWalker ogw = new ForwardWalker(disp);
@@ -109,7 +110,7 @@ public class HiveOpConverterPostProc implements Transform {
       }
       joinOp.getConf().setBaseSrc(baseSrc);
       joinOp.getConf().setRightAliases(rightAliases);
-      joinOp.getConf().setAliasToOpInfo(aliasToJoinOpInfo);
+      joinOp.getConf().setAliasToOpInfo(aliasToOpInfo);
 
       // 2. Use self alias
       Set<String> aliases = joinOp.getSchema().getTableNames();
@@ -119,7 +120,7 @@ public class HiveOpConverterPostProc implements Transform {
                 .size() + " aliases for " + joinOp.toString());
       }
       final String joinOpAlias = aliases.iterator().next();;
-      aliasToJoinOpInfo.put(joinOpAlias, joinOp);
+      aliasToOpInfo.put(joinOpAlias, joinOp);
 
       // 3. Populate other data structures
       pctx.getJoinOps().add(joinOp);
@@ -128,4 +129,29 @@ public class HiveOpConverterPostProc implements Transform {
     }
   }
 
+
+  private class TableScanAnnotate implements NodeProcessor {
+
+    @Override
+    public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
+        Object... nodeOutputs) throws SemanticException {
+      TableScanOperator tableScanOp = (TableScanOperator) nd;
+
+      // 1. Get alias from topOps
+      String opAlias = null;
+      for (Map.Entry<String, Operator<? extends OperatorDesc>> topOpEntry : pctx.getTopOps().entrySet()) {
+        if (topOpEntry.getValue() == tableScanOp) {
+          opAlias = topOpEntry.getKey();
+        }
+      }
+
+      assert opAlias != null;
+
+      // 2. Add alias to 1) aliasToOpInfo and 2) opToAlias
+      aliasToOpInfo.put(opAlias, tableScanOp);
+
+      return null;
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/4d984bde/ql/src/test/queries/clientpositive/cbo_rp_auto_join0.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/cbo_rp_auto_join0.q b/ql/src/test/queries/clientpositive/cbo_rp_auto_join0.q
index ad9ebd1..f8511b1 100644
--- a/ql/src/test/queries/clientpositive/cbo_rp_auto_join0.q
+++ b/ql/src/test/queries/clientpositive/cbo_rp_auto_join0.q
@@ -1,3 +1,4 @@
+set hive.cbo.returnpath.hiveop=true;
 set hive.stats.fetch.column.stats=true;
 set hive.auto.convert.join = true;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/4d984bde/ql/src/test/queries/clientpositive/cbo_rp_auto_join1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/cbo_rp_auto_join1.q b/ql/src/test/queries/clientpositive/cbo_rp_auto_join1.q
index c1f4352..096ae10 100644
--- a/ql/src/test/queries/clientpositive/cbo_rp_auto_join1.q
+++ b/ql/src/test/queries/clientpositive/cbo_rp_auto_join1.q
@@ -1,3 +1,4 @@
+set hive.cbo.returnpath.hiveop=true;
 set hive.stats.fetch.column.stats=true;
 set hive.enforce.bucketing = true;
 set hive.enforce.sorting = true;

http://git-wip-us.apache.org/repos/asf/hive/blob/4d984bde/ql/src/test/queries/clientpositive/cbo_rp_join0.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/cbo_rp_join0.q b/ql/src/test/queries/clientpositive/cbo_rp_join0.q
index acfff75..3e3a013 100644
--- a/ql/src/test/queries/clientpositive/cbo_rp_join0.q
+++ b/ql/src/test/queries/clientpositive/cbo_rp_join0.q
@@ -1,4 +1,5 @@
 set hive.cbo.enable=true;
+set hive.cbo.returnpath.hiveop=true;
 set hive.exec.check.crossproducts=false;
 
 set hive.stats.fetch.column.stats=true;

http://git-wip-us.apache.org/repos/asf/hive/blob/4d984bde/ql/src/test/results/clientpositive/cbo_rp_auto_join0.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/cbo_rp_auto_join0.q.out b/ql/src/test/results/clientpositive/cbo_rp_auto_join0.q.out
index 6fdc935..d1bc6d4 100644
--- a/ql/src/test/results/clientpositive/cbo_rp_auto_join0.q.out
+++ b/ql/src/test/results/clientpositive/cbo_rp_auto_join0.q.out
@@ -113,13 +113,17 @@ STAGE PLANS:
           mode: mergepartial
           outputColumnNames: _col0
           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-          File Output Operator
-            compressed: false
+          Select Operator
+            expressions: _col0 (type: bigint)
+            outputColumnNames: _col0
             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
+            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
@@ -242,13 +246,17 @@ STAGE PLANS:
           mode: mergepartial
           outputColumnNames: _col0
           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-          File Output Operator
-            compressed: false
+          Select Operator
+            expressions: _col0 (type: bigint)
+            outputColumnNames: _col0
             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
+            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

http://git-wip-us.apache.org/repos/asf/hive/blob/4d984bde/ql/src/test/results/clientpositive/cbo_rp_auto_join1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/cbo_rp_auto_join1.q.out b/ql/src/test/results/clientpositive/cbo_rp_auto_join1.q.out
index 79b4650..d52586f 100644
--- a/ql/src/test/results/clientpositive/cbo_rp_auto_join1.q.out
+++ b/ql/src/test/results/clientpositive/cbo_rp_auto_join1.q.out
@@ -95,39 +95,43 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: a
+            alias: subq1:a
             Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: key is not null (type: boolean)
               Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int)
-                outputColumnNames: _col0
+                outputColumnNames: key
                 Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                 Sorted Merge Bucket Map Join Operator
                   condition map:
                        Inner Join 0 to 1
                   keys:
-                    0 _col0 (type: int)
-                    1 _col0 (type: int)
-                  Group By Operator
-                    aggregations: count()
-                    mode: hash
-                    outputColumnNames: _col0
-                    Reduce Output Operator
-                      sort order: 
-                      value expressions: _col0 (type: bigint)
+                    0 key (type: int)
+                    1 key (type: int)
+                  Select Operator
+                    Group By Operator
+                      aggregations: count()
+                      mode: hash
+                      outputColumnNames: _col0
+                      Reduce Output Operator
+                        sort order: 
+                        value expressions: _col0 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
           mode: mergepartial
-          outputColumnNames: _col0
-          File Output Operator
-            compressed: false
-            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
+          outputColumnNames: $f0
+          Select Operator
+            expressions: $f0 (type: bigint)
+            outputColumnNames: $f0
+            File Output Operator
+              compressed: false
+              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
@@ -182,35 +186,38 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: a
+            alias: subq2:subq1:a
             Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: key is not null (type: boolean)
               Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int)
-                outputColumnNames: _col0
+                outputColumnNames: key
                 Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                 Sorted Merge Bucket Map Join Operator
                   condition map:
                        Inner Join 0 to 1
                   keys:
-                    0 _col0 (type: int)
-                    1 _col0 (type: int)
-                  outputColumnNames: _col0
-                  Group By Operator
-                    keys: _col0 (type: int)
-                    mode: hash
-                    outputColumnNames: _col0
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
+                    0 key (type: int)
+                    1 key (type: int)
+                  outputColumnNames: key
+                  Select Operator
+                    expressions: key (type: int)
+                    outputColumnNames: key
+                    Group By Operator
+                      keys: key (type: int)
+                      mode: hash
+                      outputColumnNames: _col0
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
       Reduce Operator Tree:
         Group By Operator
           keys: KEY._col0 (type: int)
           mode: mergepartial
-          outputColumnNames: _col0
+          outputColumnNames: key
           Select Operator
             Group By Operator
               aggregations: count()
@@ -234,13 +241,16 @@ STAGE PLANS:
         Group By Operator
           aggregations: count(VALUE._col0)
           mode: mergepartial
-          outputColumnNames: _col0
-          File Output Operator
-            compressed: false
-            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
+          outputColumnNames: $f0
+          Select Operator
+            expressions: $f0 (type: bigint)
+            outputColumnNames: $f0
+            File Output Operator
+              compressed: false
+              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
@@ -322,38 +332,41 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: a
+            alias: src2:subq2:a
             Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: key is not null (type: boolean)
               Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int)
-                outputColumnNames: _col0
+                outputColumnNames: key
                 Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                 Sorted Merge Bucket Map Join Operator
                   condition map:
                        Inner Join 0 to 1
                   keys:
-                    0 _col0 (type: int)
-                    1 _col0 (type: int)
-                  outputColumnNames: _col0
-                  Group By Operator
-                    aggregations: count()
-                    keys: _col0 (type: int)
-                    mode: hash
-                    outputColumnNames: _col0, _col1
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
-                      value expressions: _col1 (type: bigint)
+                    0 key (type: int)
+                    1 key (type: int)
+                  outputColumnNames: key
+                  Select Operator
+                    expressions: key (type: int)
+                    outputColumnNames: key
+                    Group By Operator
+                      aggregations: count()
+                      keys: key (type: int)
+                      mode: hash
+                      outputColumnNames: _col0, _col1
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        value expressions: _col1 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
           keys: KEY._col0 (type: int)
           mode: mergepartial
-          outputColumnNames: _col0, _col1
+          outputColumnNames: key, $f1
           File Output Operator
             compressed: false
             table:
@@ -366,27 +379,27 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             Reduce Output Operator
-              key expressions: _col0 (type: int)
+              key expressions: key (type: int)
               sort order: +
-              Map-reduce partition columns: _col0 (type: int)
-              value expressions: _col1 (type: bigint)
+              Map-reduce partition columns: key (type: int)
+              value expressions: $f1 (type: bigint)
           TableScan
             Reduce Output Operator
-              key expressions: _col0 (type: int)
+              key expressions: key (type: int)
               sort order: +
-              Map-reduce partition columns: _col0 (type: int)
-              value expressions: _col1 (type: bigint)
+              Map-reduce partition columns: key (type: int)
+              value expressions: $f1 (type: bigint)
       Reduce Operator Tree:
         Join Operator
           condition map:
                Inner Join 0 to 1
           keys:
-            0 _col0 (type: int)
-            1 _col0 (type: int)
-          outputColumnNames: _col0, _col1, _col3
+            0 key (type: int)
+            1 key (type: int)
+          outputColumnNames: key, $f1, $f10
           Select Operator
-            expressions: _col0 (type: int), _col3 (type: bigint), _col1 (type: bigint)
-            outputColumnNames: _col0, _col1, _col2
+            expressions: key (type: int), $f10 (type: bigint), $f1 (type: bigint)
+            outputColumnNames: key, cnt1, cnt11
             File Output Operator
               compressed: false
               table:
@@ -398,38 +411,41 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: a
+            alias: src1:subq1:a
             Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: key is not null (type: boolean)
               Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int)
-                outputColumnNames: _col0
+                outputColumnNames: key
                 Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
                 Sorted Merge Bucket Map Join Operator
                   condition map:
                        Inner Join 0 to 1
                   keys:
-                    0 _col0 (type: int)
-                    1 _col0 (type: int)
-                  outputColumnNames: _col0
-                  Group By Operator
-                    aggregations: count()
-                    keys: _col0 (type: int)
-                    mode: hash
-                    outputColumnNames: _col0, _col1
-                    Reduce Output Operator
-                      key expressions: _col0 (type: int)
-                      sort order: +
-                      Map-reduce partition columns: _col0 (type: int)
-                      value expressions: _col1 (type: bigint)
+                    0 key (type: int)
+                    1 key (type: int)
+                  outputColumnNames: key
+                  Select Operator
+                    expressions: key (type: int)
+                    outputColumnNames: key
+                    Group By Operator
+                      aggregations: count()
+                      keys: key (type: int)
+                      mode: hash
+                      outputColumnNames: _col0, _col1
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        value expressions: _col1 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
           keys: KEY._col0 (type: int)
           mode: mergepartial
-          outputColumnNames: _col0, _col1
+          outputColumnNames: key, $f1
           File Output Operator
             compressed: false
             table:
@@ -514,39 +530,43 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: a
+            alias: subq1:a
             Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (key < 6) (type: boolean)
               Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int)
-                outputColumnNames: _col0
+                outputColumnNames: key
                 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
                 Sorted Merge Bucket Map Join Operator
                   condition map:
                        Inner Join 0 to 1
                   keys:
-                    0 _col0 (type: int)
-                    1 _col0 (type: int)
-                  Group By Operator
-                    aggregations: count()
-                    mode: hash
-                    outputColumnNames: _col0
-                    Reduce Output Operator
-                      sort order: 
-                      value expressions: _col0 (type: bigint)
+                    0 key (type: int)
+                    1 key (type: int)
+                  Select Operator
+                    Group By Operator
+                      aggregations: count()
+                      mode: hash
+                      outputColumnNames: _col0
+                      Reduce Output Operator
+                        sort order: 
+                        value expressions: _col0 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
           mode: mergepartial
-          outputColumnNames: _col0
-          File Output Operator
-            compressed: false
-            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
+          outputColumnNames: $f0
+          Select Operator
+            expressions: $f0 (type: bigint)
+            outputColumnNames: $f0
+            File Output Operator
+              compressed: false
+              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
@@ -610,39 +630,43 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: b
+            alias: subq2:subq1:a
             Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (((key < 8) and (key < 6)) and key is not null) (type: boolean)
               Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int)
-                outputColumnNames: _col0
+                outputColumnNames: key
                 Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
                 Sorted Merge Bucket Map Join Operator
                   condition map:
                        Inner Join 0 to 1
                   keys:
-                    0 _col0 (type: int)
-                    1 _col0 (type: int)
-                  Group By Operator
-                    aggregations: count()
-                    mode: hash
-                    outputColumnNames: _col0
-                    Reduce Output Operator
-                      sort order: 
-                      value expressions: _col0 (type: bigint)
+                    0 key (type: int)
+                    1 key (type: int)
+                  Select Operator
+                    Group By Operator
+                      aggregations: count()
+                      mode: hash
+                      outputColumnNames: _col0
+                      Reduce Output Operator
+                        sort order: 
+                        value expressions: _col0 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
           mode: mergepartial
-          outputColumnNames: _col0
-          File Output Operator
-            compressed: false
-            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
+          outputColumnNames: $f0
+          Select Operator
+            expressions: $f0 (type: bigint)
+            outputColumnNames: $f0
+            File Output Operator
+              compressed: false
+              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
@@ -730,39 +754,43 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: a
+            alias: subq2:subq1:a
             Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (((key < 8) and (key < 6)) and key is not null) (type: boolean)
               Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int)
-                outputColumnNames: _col0
+                outputColumnNames: key
                 Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
                 Sorted Merge Bucket Map Join Operator
                   condition map:
                        Inner Join 0 to 1
                   keys:
-                    0 _col0 (type: int)
-                    1 _col0 (type: int)
-                  Group By Operator
-                    aggregations: count()
-                    mode: hash
-                    outputColumnNames: _col0
-                    Reduce Output Operator
-                      sort order: 
-                      value expressions: _col0 (type: bigint)
+                    0 key (type: int)
+                    1 key (type: int)
+                  Select Operator
+                    Group By Operator
+                      aggregations: count()
+                      mode: hash
+                      outputColumnNames: _col0
+                      Reduce Output Operator
+                        sort order: 
+                        value expressions: _col0 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
           mode: mergepartial
-          outputColumnNames: _col0
-          File Output Operator
-            compressed: false
-            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
+          outputColumnNames: $f0
+          Select Operator
+            expressions: $f0 (type: bigint)
+            outputColumnNames: $f0
+            File Output Operator
+              compressed: false
+              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
@@ -840,39 +868,43 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: a
+            alias: subq1:a
             Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (key < 8) (type: boolean)
               Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int)
-                outputColumnNames: _col0
+                outputColumnNames: key
                 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
                 Sorted Merge Bucket Map Join Operator
                   condition map:
                        Inner Join 0 to 1
                   keys:
-                    0 _col0 (type: int)
-                    1 _col0 (type: int)
-                  Group By Operator
-                    aggregations: count()
-                    mode: hash
-                    outputColumnNames: _col0
-                    Reduce Output Operator
-                      sort order: 
-                      value expressions: _col0 (type: bigint)
+                    0 key (type: int)
+                    1 key (type: int)
+                  Select Operator
+                    Group By Operator
+                      aggregations: count()
+                      mode: hash
+                      outputColumnNames: _col0
+                      Reduce Output Operator
+                        sort order: 
+                        value expressions: _col0 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
           mode: mergepartial
-          outputColumnNames: _col0
-          File Output Operator
-            compressed: false
-            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
+          outputColumnNames: $f0
+          Select Operator
+            expressions: $f0 (type: bigint)
+            outputColumnNames: $f0
+            File Output Operator
+              compressed: false
+              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
@@ -927,54 +959,56 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: a
+            alias: subq1:a
             Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE
-            Select Operator
-              expressions: (key + 1) (type: int)
-              outputColumnNames: _col0
-              Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-              Filter Operator
-                predicate: _col0 is not null (type: boolean)
-                Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+            Filter Operator
+              predicate: (key + 1) is not null (type: boolean)
+              Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: key (type: int)
+                outputColumnNames: key
+                Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
-                  key expressions: _col0 (type: int)
+                  key expressions: (key + 1) (type: int)
                   sort order: +
-                  Map-reduce partition columns: _col0 (type: int)
-                  Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                  Map-reduce partition columns: (key + 1) (type: int)
+                  Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
           TableScan
-            alias: a
+            alias: subq2:a
             Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE
-            Select Operator
-              expressions: (key + 1) (type: int)
-              outputColumnNames: _col0
-              Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
-              Filter Operator
-                predicate: _col0 is not null (type: boolean)
-                Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+            Filter Operator
+              predicate: (key + 1) is not null (type: boolean)
+              Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: key (type: int)
+                outputColumnNames: key
+                Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
-                  key expressions: _col0 (type: int)
+                  key expressions: (key + 1) (type: int)
                   sort order: +
-                  Map-reduce partition columns: _col0 (type: int)
-                  Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: COMPLETE
+                  Map-reduce partition columns: (key + 1) (type: int)
+                  Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
       Reduce Operator Tree:
         Join Operator
           condition map:
                Inner Join 0 to 1
           keys:
-            0 _col0 (type: int)
-            1 _col0 (type: int)
-          Statistics: Num rows: 10 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
-          Group By Operator
-            aggregations: count()
-            mode: hash
-            outputColumnNames: _col0
-            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-            File Output Operator
-              compressed: false
-              table:
-                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                  serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+            0 (key + 1) (type: int)
+            1 (key + 1) (type: int)
+          Statistics: Num rows: 5 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+          Select Operator
+            Statistics: Num rows: 5 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+            Group By Operator
+              aggregations: count()
+              mode: hash
+              outputColumnNames: _col0
+              Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+              File Output Operator
+                compressed: false
+                table:
+                    input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                    output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                    serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
 
   Stage: Stage-2
     Map Reduce
@@ -988,15 +1022,19 @@ STAGE PLANS:
         Group By Operator
           aggregations: count(VALUE._col0)
           mode: mergepartial
-          outputColumnNames: _col0
+          outputColumnNames: $f0
           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
-          File Output Operator
-            compressed: false
+          Select Operator
+            expressions: $f0 (type: bigint)
+            outputColumnNames: $f0
             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
+            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
@@ -1046,39 +1084,43 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: a
+            alias: subq1:a
             Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (key < 6) (type: boolean)
               Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int)
-                outputColumnNames: _col0
+                outputColumnNames: key
                 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
                 Sorted Merge Bucket Map Join Operator
                   condition map:
                        Inner Join 0 to 1
                   keys:
-                    0 _col0 (type: int)
-                    1 _col0 (type: int)
-                  Group By Operator
-                    aggregations: count()
-                    mode: hash
-                    outputColumnNames: _col0
-                    Reduce Output Operator
-                      sort order: 
-                      value expressions: _col0 (type: bigint)
+                    0 key (type: int)
+                    1 key (type: int)
+                  Select Operator
+                    Group By Operator
+                      aggregations: count()
+                      mode: hash
+                      outputColumnNames: _col0
+                      Reduce Output Operator
+                        sort order: 
+                        value expressions: _col0 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
           mode: mergepartial
-          outputColumnNames: _col0
-          File Output Operator
-            compressed: false
-            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
+          outputColumnNames: $f0
+          Select Operator
+            expressions: $f0 (type: bigint)
+            outputColumnNames: $f0
+            File Output Operator
+              compressed: false
+              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
@@ -1134,41 +1176,45 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: a
+            alias: subq1:a
             Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (key < 6) (type: boolean)
               Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int)
-                outputColumnNames: _col0
+                outputColumnNames: key
                 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
                 Sorted Merge Bucket Map Join Operator
                   condition map:
                        Inner Join 0 to 1
                        Inner Join 0 to 2
                   keys:
-                    0 _col0 (type: int)
-                    1 _col0 (type: int)
-                    2 _col0 (type: int)
-                  Group By Operator
-                    aggregations: count()
-                    mode: hash
-                    outputColumnNames: _col0
-                    Reduce Output Operator
-                      sort order: 
-                      value expressions: _col0 (type: bigint)
+                    0 key (type: int)
+                    1 key (type: int)
+                    2 key (type: int)
+                  Select Operator
+                    Group By Operator
+                      aggregations: count()
+                      mode: hash
+                      outputColumnNames: _col0
+                      Reduce Output Operator
+                        sort order: 
+                        value expressions: _col0 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
           mode: mergepartial
-          outputColumnNames: _col0
-          File Output Operator
-            compressed: false
-            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
+          outputColumnNames: $f0
+          Select Operator
+            expressions: $f0 (type: bigint)
+            outputColumnNames: $f0
+            File Output Operator
+              compressed: false
+              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
@@ -1240,39 +1286,43 @@ STAGE PLANS:
     Map Reduce
       Map Operator Tree:
           TableScan
-            alias: b
+            alias: a:subq2:subq1:a
             Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE
             Filter Operator
               predicate: (((key < 8) and (key < 6)) and key is not null) (type: boolean)
               Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int)
-                outputColumnNames: _col0
+                outputColumnNames: key
                 Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
                 Sorted Merge Bucket Map Join Operator
                   condition map:
                        Inner Join 0 to 1
                   keys:
-                    0 _col0 (type: int)
-                    1 _col0 (type: int)
-                  Group By Operator
-                    aggregations: count()
-                    mode: hash
-                    outputColumnNames: _col0
-                    Reduce Output Operator
-                      sort order: 
-                      value expressions: _col0 (type: bigint)
+                    0 key (type: int)
+                    1 key (type: int)
+                  Select Operator
+                    Group By Operator
+                      aggregations: count()
+                      mode: hash
+                      outputColumnNames: _col0
+                      Reduce Output Operator
+                        sort order: 
+                        value expressions: _col0 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
           mode: mergepartial
-          outputColumnNames: _col0
-          File Output Operator
-            compressed: false
-            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
+          outputColumnNames: $f0
+          Select Operator
+            expressions: $f0 (type: bigint)
+            outputColumnNames: $f0
+            File Output Operator
+              compressed: false
+              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
@@ -1335,18 +1385,18 @@ STAGE PLANS:
               Statistics: Num rows: 10 Data size: 930 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int), value (type: string)
-                outputColumnNames: _col0, _col1
+                outputColumnNames: key, value
                 Statistics: Num rows: 10 Data size: 930 Basic stats: COMPLETE Column stats: COMPLETE
                 Sorted Merge Bucket Map Join Operator
                   condition map:
                        Inner Join 0 to 1
                   keys:
-                    0 _col0 (type: int)
-                    1 _col0 (type: int)
-                  outputColumnNames: _col0, _col1, _col3
+                    0 key (type: int)
+                    1 key (type: int)
+                  outputColumnNames: key, value, value0
                   Select Operator
-                    expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string)
-                    outputColumnNames: _col0, _col1, _col2
+                    expressions: key (type: int), value (type: string), value0 (type: string)
+                    outputColumnNames: key, val1, val2
                     File Output Operator
                       compressed: false
                       table:
@@ -1416,18 +1466,18 @@ STAGE PLANS:
               Statistics: Num rows: 10 Data size: 930 Basic stats: COMPLETE Column stats: COMPLETE
               Select Operator
                 expressions: key (type: int), value (type: string)
-                outputColumnNames: _col0, _col1
+                outputColumnNames: key, value
                 Statistics: Num rows: 10 Data size: 930 Basic stats: COMPLETE Column stats: COMPLETE
                 Sorted Merge Bucket Map Join Operator
                   condition map:
                        Inner Join 0 to 1
                   keys:
-                    0 _col0 (type: int)
-                    1 _col0 (type: int)
-                  outputColumnNames: _col0, _col1, _col3
+                    0 key (type: int)
+                    1 key (type: int)
+                  outputColumnNames: key, value, value0
                   Select Operator
-                    expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string)
-                    outputColumnNames: _col0, _col1, _col2
+                    expressions: key (type: int), value (type: string), value0 (type: string)
+                    outputColumnNames: key, val1, val2
                     File Output Operator
                       compressed: false
                       table:

http://git-wip-us.apache.org/repos/asf/hive/blob/4d984bde/ql/src/test/results/clientpositive/cbo_rp_join0.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/cbo_rp_join0.q.out b/ql/src/test/results/clientpositive/cbo_rp_join0.q.out
index 1894110..93fed08 100644
--- a/ql/src/test/results/clientpositive/cbo_rp_join0.q.out
+++ b/ql/src/test/results/clientpositive/cbo_rp_join0.q.out
@@ -21,40 +21,46 @@ STAGE PLANS:
           TableScan
             alias: cbo_t1
             Statistics: Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
-            Select Operator
-              expressions: key (type: string), c_int (type: int)
-              outputColumnNames: _col0, _col1
-              Statistics: Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                key expressions: _col0 (type: string)
-                sort order: +
-                Map-reduce partition columns: _col0 (type: string)
-                Statistics: Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
-                value expressions: _col1 (type: int)
+            Filter Operator
+              predicate: key is not null (type: boolean)
+              Statistics: Num rows: 18 Data size: 1424 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: key (type: string), c_int (type: int)
+                outputColumnNames: key, c_int
+                Statistics: Num rows: 18 Data size: 1424 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: key (type: string)
+                  sort order: +
+                  Map-reduce partition columns: key (type: string)
+                  Statistics: Num rows: 18 Data size: 1424 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: c_int (type: int)
           TableScan
-            alias: cbo_t2
+            alias: cbo_t2:cbo_t2
             Statistics: Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
-            Select Operator
-              expressions: key (type: string), c_int (type: int)
-              outputColumnNames: _col0, _col1
-              Statistics: Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                key expressions: _col0 (type: string)
-                sort order: +
-                Map-reduce partition columns: _col0 (type: string)
-                Statistics: Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
-                value expressions: _col1 (type: int)
+            Filter Operator
+              predicate: key is not null (type: boolean)
+              Statistics: Num rows: 18 Data size: 1424 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: key (type: string), c_int (type: int)
+                outputColumnNames: key, c_int
+                Statistics: Num rows: 18 Data size: 1424 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: key (type: string)
+                  sort order: +
+                  Map-reduce partition columns: key (type: string)
+                  Statistics: Num rows: 18 Data size: 1424 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: c_int (type: int)
           TableScan
-            alias: cbo_t3
+            alias: cbo_t3:cbo_t3
             Statistics: Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 20 Data size: 1530 Basic stats: COMPLETE Column stats: COMPLETE
               Reduce Output Operator
-                key expressions: _col0 (type: string)
+                key expressions: key (type: string)
                 sort order: +
-                Map-reduce partition columns: _col0 (type: string)
+                Map-reduce partition columns: key (type: string)
                 Statistics: Num rows: 20 Data size: 1530 Basic stats: COMPLETE Column stats: COMPLETE
       Reduce Operator Tree:
         Join Operator
@@ -62,18 +68,22 @@ STAGE PLANS:
                Inner Join 0 to 1
                Right Outer Join0 to 2
           keys:
-            0 _col0 (type: string)
-            1 _col0 (type: string)
-            2 _col0 (type: string)
-          outputColumnNames: _col0, _col1, _col2, _col3
-          Statistics: Num rows: 400 Data size: 71200 Basic stats: COMPLETE Column stats: COMPLETE
-          File Output Operator
-            compressed: false
-            Statistics: Num rows: 400 Data size: 71200 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
+            0 key (type: string)
+            1 key (type: string)
+            2 key (type: string)
+          outputColumnNames: key, c_int, key0, c_int0
+          Statistics: Num rows: 324 Data size: 57672 Basic stats: COMPLETE Column stats: COMPLETE
+          Select Operator
+            expressions: key (type: string), c_int (type: int), key0 (type: string), c_int0 (type: int)
+            outputColumnNames: key, c_int, p, q
+            Statistics: Num rows: 324 Data size: 57672 Basic stats: COMPLETE Column stats: COMPLETE
+            File Output Operator
+              compressed: false
+              Statistics: Num rows: 324 Data size: 57672 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
@@ -660,54 +670,63 @@ STAGE PLANS:
           TableScan
             alias: cbo_t1
             Statistics: Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
-            Select Operator
-              expressions: key (type: string), c_int (type: int)
-              outputColumnNames: _col0, _col1
-              Statistics: Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                key expressions: _col0 (type: string)
-                sort order: +
-                Map-reduce partition columns: _col0 (type: string)
-                Statistics: Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
-                value expressions: _col1 (type: int)
+            Filter Operator
+              predicate: key is not null (type: boolean)
+              Statistics: Num rows: 18 Data size: 1424 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: key (type: string), c_int (type: int)
+                outputColumnNames: key, c_int
+                Statistics: Num rows: 18 Data size: 1424 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: key (type: string)
+                  sort order: +
+                  Map-reduce partition columns: key (type: string)
+                  Statistics: Num rows: 18 Data size: 1424 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: c_int (type: int)
           TableScan
-            alias: cbo_t2
+            alias: cbo_t2:cbo_t2
             Statistics: Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
-            Select Operator
-              expressions: key (type: string), c_int (type: int)
-              outputColumnNames: _col0, _col1
-              Statistics: Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                key expressions: _col0 (type: string)
-                sort order: +
-                Map-reduce partition columns: _col0 (type: string)
-                Statistics: Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
-                value expressions: _col1 (type: int)
+            Filter Operator
+              predicate: key is not null (type: boolean)
+              Statistics: Num rows: 18 Data size: 1424 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: key (type: string), c_int (type: int)
+                outputColumnNames: key, c_int
+                Statistics: Num rows: 18 Data size: 1424 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: key (type: string)
+                  sort order: +
+                  Map-reduce partition columns: key (type: string)
+                  Statistics: Num rows: 18 Data size: 1424 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: c_int (type: int)
           TableScan
-            alias: cbo_t3
+            alias: cbo_t3:cbo_t3
             Statistics: Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
             Select Operator
               expressions: key (type: string)
-              outputColumnNames: _col0
+              outputColumnNames: key
               Statistics: Num rows: 20 Data size: 1530 Basic stats: COMPLETE Column stats: COMPLETE
               Reduce Output Operator
-                key expressions: _col0 (type: string)
+                key expressions: key (type: string)
                 sort order: +
-                Map-reduce partition columns: _col0 (type: string)
+                Map-reduce partition columns: key (type: string)
                 Statistics: Num rows: 20 Data size: 1530 Basic stats: COMPLETE Column stats: COMPLETE
           TableScan
-            alias: cbo_t1
+            alias: cbo_t4:cbo_t1
             Statistics: Num rows: 20 Data size: 262 Basic stats: COMPLETE Column stats: COMPLETE
-            Select Operator
-              expressions: key (type: string), c_int (type: int)
-              outputColumnNames: _col0, _col1
-              Statistics: Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
-              Reduce Output Operator
-                key expressions: _col0 (type: string)
-                sort order: +
-                Map-reduce partition columns: _col0 (type: string)
-                Statistics: Num rows: 20 Data size: 1602 Basic stats: COMPLETE Column stats: COMPLETE
-                value expressions: _col1 (type: int)
+            Filter Operator
+              predicate: key is not null (type: boolean)
+              Statistics: Num rows: 18 Data size: 1424 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                expressions: key (type: string), c_int (type: int)
+                outputColumnNames: key, c_int
+                Statistics: Num rows: 18 Data size: 1424 Basic stats: COMPLETE Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: key (type: string)
+                  sort order: +
+                  Map-reduce partition columns: key (type: string)
+                  Statistics: Num rows: 18 Data size: 1424 Basic stats: COMPLETE Column stats: COMPLETE
+                  value expressions: c_int (type: int)
       Reduce Operator Tree:
         Join Operator
           condition map:
@@ -715,19 +734,19 @@ STAGE PLANS:
                Right Outer Join0 to 2
                Left Outer Join0 to 3
           keys:
-            0 _col0 (type: string)
-            1 _col0 (type: string)
-            2 _col0 (type: string)
-            3 _col0 (type: string)
-          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col6
-          Statistics: Num rows: 2000 Data size: 534000 Basic stats: COMPLETE Column stats: COMPLETE
+            0 key (type: string)
+            1 key (type: string)
+            2 key (type: string)
+            3 key (type: string)
+          outputColumnNames: key, c_int, key0, c_int0, key1, c_int1
+          Statistics: Num rows: 1458 Data size: 389286 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
-            expressions: _col0 (type: string), _col1 (type: int), _col2 (type: string), _col3 (type: int), _col4 (type: string), _col6 (type: int)
-            outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-            Statistics: Num rows: 2000 Data size: 534000 Basic stats: COMPLETE Column stats: COMPLETE
+            expressions: key (type: string), c_int (type: int), key0 (type: string), c_int0 (type: int), key1 (type: string), c_int1 (type: int)
+            outputColumnNames: key, c_int, p, q, x, b
+            Statistics: Num rows: 1458 Data size: 389286 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
               compressed: false
-              Statistics: Num rows: 2000 Data size: 534000 Basic stats: COMPLETE Column stats: COMPLETE
+              Statistics: Num rows: 1458 Data size: 389286 Basic stats: COMPLETE Column stats: COMPLETE
               table:
                   input format: org.apache.hadoop.mapred.TextInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat


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

Posted by ga...@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/hbase-metastore
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


[05/50] [abbrv] hive git commit: HIVE-11130 - Refactoring the code so that HiveTxnManager interface will support lock/unlock table/database object (Aihua Xu, reviewed by Alan Gates and Chao Sun)

Posted by ga...@apache.org.
HIVE-11130 - Refactoring the code so that HiveTxnManager interface will support lock/unlock table/database object (Aihua Xu, reviewed by Alan Gates and Chao Sun)


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

Branch: refs/heads/hbase-metastore
Commit: 46c76d6b8e1fb323d1f8ac75061b6fd3bb747d6d
Parents: 999e0e3
Author: Aihua Xu <ai...@gmail.com>
Authored: Tue Jul 14 11:49:09 2015 -0700
Committer: Chao Sun <ch...@cloudera.com>
Committed: Tue Jul 14 11:49:09 2015 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java | 147 +------------------
 .../hadoop/hive/ql/lockmgr/HiveLockObject.java  |  35 +++++
 .../hadoop/hive/ql/lockmgr/HiveTxnManager.java  |  47 +++++-
 .../hive/ql/lockmgr/HiveTxnManagerImpl.java     | 140 ++++++++++++++++++
 4 files changed, 227 insertions(+), 142 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/46c76d6b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
index 049857b..a8c6aca 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
@@ -2444,8 +2444,8 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
         locks = lockMgr.getLocks(false, isExt);
       }
       else {
-        locks = lockMgr.getLocks(getHiveObject(showLocks.getTableName(),
-            showLocks.getPartSpec()),
+        locks = lockMgr.getLocks(HiveLockObject.createFrom(db,
+            showLocks.getTableName(), showLocks.getPartSpec()),
             true, isExt);
       }
 
@@ -2705,46 +2705,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
   private int lockTable(LockTableDesc lockTbl) throws HiveException {
     Context ctx = driverContext.getCtx();
     HiveTxnManager txnManager = ctx.getHiveTxnManager();
-    if (!txnManager.supportsExplicitLock()) {
-      throw new HiveException(ErrorMsg.LOCK_REQUEST_UNSUPPORTED,
-          conf.getVar(HiveConf.ConfVars.HIVE_TXN_MANAGER));
-    }
-    HiveLockManager lockMgr = txnManager.getLockManager();
-    if (lockMgr == null) {
-      throw new HiveException("lock Table LockManager not specified");
-    }
-
-    HiveLockMode mode = HiveLockMode.valueOf(lockTbl.getMode());
-    String tabName = lockTbl.getTableName();
-    Table  tbl = db.getTable(tabName);
-    if (tbl == null) {
-      throw new HiveException("Table " + tabName + " does not exist ");
-    }
-
-    Map<String, String> partSpec = lockTbl.getPartSpec();
-    HiveLockObjectData lockData =
-        new HiveLockObjectData(lockTbl.getQueryId(),
-            String.valueOf(System.currentTimeMillis()),
-            "EXPLICIT",
-            lockTbl.getQueryStr());
-
-    if (partSpec == null) {
-      HiveLock lck = lockMgr.lock(new HiveLockObject(tbl, lockData), mode, true);
-      if (lck == null) {
-        return 1;
-      }
-      return 0;
-    }
-
-    Partition par = db.getPartition(tbl, partSpec, false);
-    if (par == null) {
-      throw new HiveException("Partition " + partSpec + " for table " + tabName + " does not exist");
-    }
-    HiveLock lck = lockMgr.lock(new HiveLockObject(par, lockData), mode, true);
-    if (lck == null) {
-      return 1;
-    }
-    return 0;
+    return txnManager.lockTable(db, lockTbl);
   }
 
   /**
@@ -2759,33 +2720,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
   private int lockDatabase(LockDatabaseDesc lockDb) throws HiveException {
     Context ctx = driverContext.getCtx();
     HiveTxnManager txnManager = ctx.getHiveTxnManager();
-    if (!txnManager.supportsExplicitLock()) {
-      throw new HiveException(ErrorMsg.LOCK_REQUEST_UNSUPPORTED,
-          conf.getVar(HiveConf.ConfVars.HIVE_TXN_MANAGER));
-    }
-    HiveLockManager lockMgr = txnManager.getLockManager();
-    if (lockMgr == null) {
-      throw new HiveException("lock Database LockManager not specified");
-    }
-
-    HiveLockMode mode = HiveLockMode.valueOf(lockDb.getMode());
-    String dbName = lockDb.getDatabaseName();
-
-    Database dbObj = db.getDatabase(dbName);
-    if (dbObj == null) {
-      throw new HiveException("Database " + dbName + " does not exist ");
-    }
-
-    HiveLockObjectData lockData =
-        new HiveLockObjectData(lockDb.getQueryId(),
-            String.valueOf(System.currentTimeMillis()),
-            "EXPLICIT", lockDb.getQueryStr());
-
-    HiveLock lck = lockMgr.lock(new HiveLockObject(dbObj.getName(), lockData), mode, true);
-    if (lck == null) {
-      return 1;
-    }
-    return 0;
+    return txnManager.lockDatabase(db, lockDb);
   }
 
   /**
@@ -2800,55 +2735,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
   private int unlockDatabase(UnlockDatabaseDesc unlockDb) throws HiveException {
     Context ctx = driverContext.getCtx();
     HiveTxnManager txnManager = ctx.getHiveTxnManager();
-    if (!txnManager.supportsExplicitLock()) {
-      throw new HiveException(ErrorMsg.LOCK_REQUEST_UNSUPPORTED,
-          conf.getVar(HiveConf.ConfVars.HIVE_TXN_MANAGER));
-    }
-    HiveLockManager lockMgr = txnManager.getLockManager();
-    if (lockMgr == null) {
-      throw new HiveException("unlock Database LockManager not specified");
-    }
-
-    String dbName = unlockDb.getDatabaseName();
-
-    Database dbObj = db.getDatabase(dbName);
-    if (dbObj == null) {
-      throw new HiveException("Database " + dbName + " does not exist ");
-    }
-    HiveLockObject obj = new HiveLockObject(dbObj.getName(), null);
-
-    List<HiveLock> locks = lockMgr.getLocks(obj, false, false);
-    if ((locks == null) || (locks.isEmpty())) {
-      throw new HiveException("Database " + dbName + " is not locked ");
-    }
-
-    for (HiveLock lock: locks) {
-      lockMgr.unlock(lock);
-
-    }
-    return 0;
-  }
-
-  private HiveLockObject getHiveObject(String tabName,
-      Map<String, String> partSpec) throws HiveException {
-    Table  tbl = db.getTable(tabName);
-    if (tbl == null) {
-      throw new HiveException("Table " + tabName + " does not exist ");
-    }
-
-    HiveLockObject obj = null;
-
-    if  (partSpec == null) {
-      obj = new HiveLockObject(tbl, null);
-    }
-    else {
-      Partition par = db.getPartition(tbl, partSpec, false);
-      if (par == null) {
-        throw new HiveException("Partition " + partSpec + " for table " + tabName + " does not exist");
-      }
-      obj = new HiveLockObject(par, null);
-    }
-    return obj;
+    return txnManager.unlockDatabase(db, unlockDb);
   }
 
   /**
@@ -2863,29 +2750,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
   private int unlockTable(UnlockTableDesc unlockTbl) throws HiveException {
     Context ctx = driverContext.getCtx();
     HiveTxnManager txnManager = ctx.getHiveTxnManager();
-    if (!txnManager.supportsExplicitLock()) {
-      throw new HiveException(ErrorMsg.LOCK_REQUEST_UNSUPPORTED,
-          conf.getVar(HiveConf.ConfVars.HIVE_TXN_MANAGER));
-    }
-    HiveLockManager lockMgr = txnManager.getLockManager();
-    if (lockMgr == null) {
-      throw new HiveException("unlock Table LockManager not specified");
-    }
-
-    String tabName = unlockTbl.getTableName();
-    HiveLockObject obj = getHiveObject(tabName, unlockTbl.getPartSpec());
-
-    List<HiveLock> locks = lockMgr.getLocks(obj, false, false);
-    if ((locks == null) || (locks.isEmpty())) {
-      throw new HiveException("Table " + tabName + " is not locked ");
-    }
-    Iterator<HiveLock> locksIter = locks.iterator();
-    while (locksIter.hasNext()) {
-      HiveLock lock = locksIter.next();
-      lockMgr.unlock(lock);
-    }
-
-    return 0;
+    return txnManager.unlockTable(db, unlockTbl);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/46c76d6b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveLockObject.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveLockObject.java b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveLockObject.java
index 7e93387..fadd074 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveLockObject.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveLockObject.java
@@ -19,9 +19,12 @@
 package org.apache.hadoop.hive.ql.lockmgr;
 
 import java.util.Arrays;
+import java.util.Map;
 
 import org.apache.commons.lang.builder.HashCodeBuilder;
 import org.apache.hadoop.hive.ql.metadata.DummyPartition;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.metadata.Table;
 
@@ -196,6 +199,38 @@ public class HiveLockObject {
     this(new String[] {par.getName()}, lockData);
   }
 
+  /**
+   * Creates a locking object for a table (when partition spec is not provided)
+   * or a table partition
+   * @param hiveDB    an object to communicate with the metastore
+   * @param tableName the table to create the locking object on
+   * @param partSpec  the spec of a partition to create the locking object on
+   * @return  the locking object
+   * @throws HiveException
+   */
+  public static HiveLockObject createFrom(Hive hiveDB, String tableName,
+      Map<String, String> partSpec) throws HiveException {
+    Table  tbl = hiveDB.getTable(tableName);
+    if (tbl == null) {
+      throw new HiveException("Table " + tableName + " does not exist ");
+    }
+
+    HiveLockObject obj = null;
+
+    if  (partSpec == null) {
+      obj = new HiveLockObject(tbl, null);
+    }
+    else {
+      Partition par = hiveDB.getPartition(tbl, partSpec, false);
+      if (par == null) {
+        throw new HiveException("Partition " + partSpec + " for table " +
+            tableName + " does not exist");
+      }
+      obj = new HiveLockObject(par, null);
+    }
+    return obj;
+  }
+
   public String[] getPaths() {
     return pathNames;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/46c76d6b/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 6c3dc33..c900548 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
@@ -18,9 +18,14 @@
 package org.apache.hadoop.hive.ql.lockmgr;
 
 import org.apache.hadoop.hive.common.ValidTxnList;
-import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.QueryPlan;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.LockDatabaseDesc;
+import org.apache.hadoop.hive.ql.plan.LockTableDesc;
+import org.apache.hadoop.hive.ql.plan.UnlockDatabaseDesc;
+import org.apache.hadoop.hive.ql.plan.UnlockTableDesc;
 
 /**
  * An interface that allows Hive to manage transactions.  All classes
@@ -116,6 +121,46 @@ public interface HiveTxnManager {
   boolean supportsExplicitLock();
 
   /**
+   * This function is called to lock the table when explicit lock command is
+   * issued on a table.
+   * @param hiveDB    an object to communicate with the metastore
+   * @param lockTbl   table locking info, such as table name, locking mode
+   * @return 0 if the locking succeeds, 1 otherwise.
+   * @throws HiveException
+   */
+  int lockTable(Hive hiveDB, LockTableDesc lockTbl) throws HiveException;
+
+  /**
+   * This function is called to unlock the table when explicit unlock command is
+   * issued on a table.
+   * @param hiveDB    an object to communicate with the metastore
+   * @param unlockTbl table unlocking info, such as table name
+   * @return 0 if the locking succeeds, 1 otherwise.
+   * @throws HiveException
+   */
+  int unlockTable(Hive hiveDB, UnlockTableDesc unlockTbl) throws HiveException;
+
+  /**
+   * This function is called to lock the database when explicit lock command is
+   * issued on a database.
+   * @param hiveDB    an object to communicate with the metastore
+   * @param lockDb    database locking info, such as database name, locking mode
+   * @return 0 if the locking succeeds, 1 otherwise.
+   * @throws HiveException
+   */
+  int lockDatabase(Hive hiveDB, LockDatabaseDesc lockDb) throws HiveException;
+
+  /**
+   * This function is called to unlock the database when explicit unlock command
+   * is issued on a database.
+   * @param hiveDB    an object to communicate with the metastore
+   * @param unlockDb  database unlocking info, such as database name
+   * @return 0 if the locking succeeds, 1 otherwise.
+   * @throws HiveException
+   */
+  int unlockDatabase(Hive hiveDB, UnlockDatabaseDesc unlockDb) throws HiveException;
+
+  /**
    * Indicate whether this transaction manager returns information about locks in the new format
    * for show locks or the old one.
    * @return true if the new format should be used.

http://git-wip-us.apache.org/repos/asf/hive/blob/46c76d6b/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 eccb8d1..ceeae68 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
@@ -17,7 +17,22 @@
  */
 package org.apache.hadoop.hive.ql.lockmgr;
 
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.ql.ErrorMsg;
+import org.apache.hadoop.hive.ql.lockmgr.HiveLockObject.HiveLockObjectData;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.metadata.Partition;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.plan.LockDatabaseDesc;
+import org.apache.hadoop.hive.ql.plan.LockTableDesc;
+import org.apache.hadoop.hive.ql.plan.UnlockDatabaseDesc;
+import org.apache.hadoop.hive.ql.plan.UnlockTableDesc;
 
 /**
  * An implementation HiveTxnManager that includes internal methods that all
@@ -44,4 +59,129 @@ abstract class HiveTxnManagerImpl implements HiveTxnManager {
     destruct();
   }
 
+  @Override
+  public int lockTable(Hive db, LockTableDesc lockTbl) throws HiveException {
+    HiveLockManager lockMgr = getAndCheckLockManager();
+
+    HiveLockMode mode = HiveLockMode.valueOf(lockTbl.getMode());
+    String tabName = lockTbl.getTableName();
+    Table  tbl = db.getTable(tabName);
+    if (tbl == null) {
+      throw new HiveException("Table " + tabName + " does not exist ");
+    }
+
+    Map<String, String> partSpec = lockTbl.getPartSpec();
+    HiveLockObjectData lockData =
+        new HiveLockObjectData(lockTbl.getQueryId(),
+            String.valueOf(System.currentTimeMillis()),
+            "EXPLICIT",
+            lockTbl.getQueryStr());
+
+    if (partSpec == null) {
+      HiveLock lck = lockMgr.lock(new HiveLockObject(tbl, lockData), mode, true);
+      if (lck == null) {
+        return 1;
+      }
+      return 0;
+    }
+
+    Partition par = db.getPartition(tbl, partSpec, false);
+    if (par == null) {
+      throw new HiveException("Partition " + partSpec + " for table " +
+          tabName + " does not exist");
+    }
+    HiveLock lck = lockMgr.lock(new HiveLockObject(par, lockData), mode, true);
+    if (lck == null) {
+      return 1;
+    }
+    return 0;
+  }
+
+  @Override
+  public int unlockTable(Hive hiveDB, UnlockTableDesc unlockTbl) throws HiveException {
+    HiveLockManager lockMgr = getAndCheckLockManager();
+
+    String tabName = unlockTbl.getTableName();
+    HiveLockObject obj = HiveLockObject.createFrom(hiveDB, tabName,
+        unlockTbl.getPartSpec());
+
+    List<HiveLock> locks = lockMgr.getLocks(obj, false, false);
+    if ((locks == null) || (locks.isEmpty())) {
+      throw new HiveException("Table " + tabName + " is not locked ");
+    }
+    Iterator<HiveLock> locksIter = locks.iterator();
+    while (locksIter.hasNext()) {
+      HiveLock lock = locksIter.next();
+      lockMgr.unlock(lock);
+    }
+
+    return 0;
+  }
+
+  @Override
+  public int lockDatabase(Hive hiveDB, LockDatabaseDesc lockDb) throws HiveException {
+    HiveLockManager lockMgr = getAndCheckLockManager();
+
+    HiveLockMode mode = HiveLockMode.valueOf(lockDb.getMode());
+    String dbName = lockDb.getDatabaseName();
+
+    Database dbObj = hiveDB.getDatabase(dbName);
+    if (dbObj == null) {
+      throw new HiveException("Database " + dbName + " does not exist ");
+    }
+
+    HiveLockObjectData lockData =
+        new HiveLockObjectData(lockDb.getQueryId(),
+            String.valueOf(System.currentTimeMillis()),
+            "EXPLICIT", lockDb.getQueryStr());
+
+    HiveLock lck = lockMgr.lock(new HiveLockObject(dbObj.getName(), lockData), mode, true);
+    if (lck == null) {
+      return 1;
+    }
+    return 0;
+  }
+
+  @Override
+  public int unlockDatabase(Hive hiveDB, UnlockDatabaseDesc unlockDb) throws HiveException {
+    HiveLockManager lockMgr = getAndCheckLockManager();
+
+    String dbName = unlockDb.getDatabaseName();
+
+    Database dbObj = hiveDB.getDatabase(dbName);
+    if (dbObj == null) {
+      throw new HiveException("Database " + dbName + " does not exist ");
+    }
+    HiveLockObject obj = new HiveLockObject(dbObj.getName(), null);
+
+    List<HiveLock> locks = lockMgr.getLocks(obj, false, false);
+    if ((locks == null) || (locks.isEmpty())) {
+      throw new HiveException("Database " + dbName + " is not locked ");
+    }
+
+    for (HiveLock lock: locks) {
+      lockMgr.unlock(lock);
+
+    }
+    return 0;
+  }
+
+  /**
+   * Gets the lock manager and verifies if the explicit lock is supported
+   * @return  the lock manager
+   * @throws HiveException
+   */
+  protected HiveLockManager getAndCheckLockManager() throws HiveException {
+    HiveLockManager lockMgr = getLockManager();
+    if (lockMgr == null) {
+      throw new HiveException("LockManager cannot be acquired");
+    }
+
+    if (!supportsExplicitLock()) {
+      throw new HiveException(ErrorMsg.LOCK_REQUEST_UNSUPPORTED,
+          conf.getVar(HiveConf.ConfVars.HIVE_TXN_MANAGER));
+    }
+
+    return lockMgr;
+  }
 }


[17/50] [abbrv] hive git commit: HIVE-11145 Remove OFFLINE and NO_DROP from tables and partitions (gates, reviewed by Ashutosh Chauhan)

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/beelinepositive/stats7.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/beelinepositive/stats7.q.out b/ql/src/test/results/beelinepositive/stats7.q.out
index 57870c5..e375cbf 100644
--- a/ql/src/test/results/beelinepositive/stats7.q.out
+++ b/ql/src/test/results/beelinepositive/stats7.q.out
@@ -61,7 +61,6 @@ No rows selected
 'Table:              ','analyze_srcpart     ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats7.db/analyze_srcpart/ds=2008-04-08/hr=11',''
 'Partition Parameters:','',''
 '','numFiles            ','1                   '
@@ -100,7 +99,6 @@ No rows selected
 'Table:              ','analyze_srcpart     ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats7.db/analyze_srcpart/ds=2008-04-08/hr=12',''
 'Partition Parameters:','',''
 '','numFiles            ','1                   '
@@ -139,7 +137,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats7.db/analyze_srcpart',''
 'Table Type:         ','MANAGED_TABLE       ',''

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/beelinepositive/stats8.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/beelinepositive/stats8.q.out b/ql/src/test/results/beelinepositive/stats8.q.out
index 017045c..1593bba 100644
--- a/ql/src/test/results/beelinepositive/stats8.q.out
+++ b/ql/src/test/results/beelinepositive/stats8.q.out
@@ -59,7 +59,6 @@ No rows selected
 'Table:              ','analyze_srcpart     ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats8.db/analyze_srcpart/ds=2008-04-08/hr=11',''
 'Partition Parameters:','',''
 '','numFiles            ','1                   '
@@ -97,7 +96,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats8.db/analyze_srcpart',''
 'Table Type:         ','MANAGED_TABLE       ',''
@@ -165,7 +163,6 @@ No rows selected
 'Table:              ','analyze_srcpart     ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats8.db/analyze_srcpart/ds=2008-04-08/hr=12',''
 'Partition Parameters:','',''
 '','numFiles            ','1                   '
@@ -230,7 +227,6 @@ No rows selected
 'Table:              ','analyze_srcpart     ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats8.db/analyze_srcpart/ds=2008-04-09/hr=11',''
 'Partition Parameters:','',''
 '','numFiles            ','1                   '
@@ -295,7 +291,6 @@ No rows selected
 'Table:              ','analyze_srcpart     ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats8.db/analyze_srcpart/ds=2008-04-09/hr=12',''
 'Partition Parameters:','',''
 '','numFiles            ','1                   '
@@ -361,7 +356,6 @@ No rows selected
 'Table:              ','analyze_srcpart     ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats8.db/analyze_srcpart/ds=2008-04-08/hr=11',''
 'Partition Parameters:','',''
 '','numFiles            ','1                   '
@@ -400,7 +394,6 @@ No rows selected
 'Table:              ','analyze_srcpart     ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats8.db/analyze_srcpart/ds=2008-04-08/hr=12',''
 'Partition Parameters:','',''
 '','numFiles            ','1                   '
@@ -439,7 +432,6 @@ No rows selected
 'Table:              ','analyze_srcpart     ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats8.db/analyze_srcpart/ds=2008-04-09/hr=11',''
 'Partition Parameters:','',''
 '','numFiles            ','1                   '
@@ -478,7 +470,6 @@ No rows selected
 'Table:              ','analyze_srcpart     ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats8.db/analyze_srcpart/ds=2008-04-09/hr=12',''
 'Partition Parameters:','',''
 '','numFiles            ','1                   '
@@ -516,7 +507,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats8.db/analyze_srcpart',''
 'Table Type:         ','MANAGED_TABLE       ',''

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/beelinepositive/stats9.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/beelinepositive/stats9.q.out b/ql/src/test/results/beelinepositive/stats9.q.out
index 8697d61..852d816 100644
--- a/ql/src/test/results/beelinepositive/stats9.q.out
+++ b/ql/src/test/results/beelinepositive/stats9.q.out
@@ -48,7 +48,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats9.db/analyze_srcbucket',''
 'Table Type:         ','MANAGED_TABLE       ',''

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/beelinepositive/stats_empty_partition.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/beelinepositive/stats_empty_partition.q.out b/ql/src/test/results/beelinepositive/stats_empty_partition.q.out
index ab6839e..415cf90 100644
--- a/ql/src/test/results/beelinepositive/stats_empty_partition.q.out
+++ b/ql/src/test/results/beelinepositive/stats_empty_partition.q.out
@@ -33,7 +33,6 @@ No rows selected
 'Table:              ','tmptable            ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats_empty_partition.db/tmptable/part=1',''
 'Partition Parameters:','',''
 '','numFiles            ','1                   '

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientnegative/alter_file_format.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/alter_file_format.q.out b/ql/src/test/results/clientnegative/alter_file_format.q.out
index d0c470b..96f1bfb 100644
--- a/ql/src/test/results/clientnegative/alter_file_format.q.out
+++ b/ql/src/test/results/clientnegative/alter_file_format.q.out
@@ -20,7 +20,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientnegative/alter_view_as_select_with_partition.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/alter_view_as_select_with_partition.q.out b/ql/src/test/results/clientnegative/alter_view_as_select_with_partition.q.out
index 447dc3a..1cbfd75 100644
--- a/ql/src/test/results/clientnegative/alter_view_as_select_with_partition.q.out
+++ b/ql/src/test/results/clientnegative/alter_view_as_select_with_partition.q.out
@@ -48,7 +48,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 Table Type:         	VIRTUAL_VIEW        	 
 Table Parameters:	 	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientnegative/stats_partialscan_autogether.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/stats_partialscan_autogether.q.out b/ql/src/test/results/clientnegative/stats_partialscan_autogether.q.out
index 321ebe5..d03c249 100644
--- a/ql/src/test/results/clientnegative/stats_partialscan_autogether.q.out
+++ b/ql/src/test/results/clientnegative/stats_partialscan_autogether.q.out
@@ -65,8 +65,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart_partial_scan	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/alter_file_format.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/alter_file_format.q.out b/ql/src/test/results/clientpositive/alter_file_format.q.out
index 4d6389a..c9e88f8 100644
--- a/ql/src/test/results/clientpositive/alter_file_format.q.out
+++ b/ql/src/test/results/clientpositive/alter_file_format.q.out
@@ -20,7 +20,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -59,7 +58,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -104,7 +102,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -149,7 +146,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -194,7 +190,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -239,7 +234,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -284,7 +278,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -349,8 +342,6 @@ Partition Value:    	[2010]
 Database:           	default             	 
 Table:              	alter_partition_format_test	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 #### A masked pattern was here ####
 	 	 
@@ -394,8 +385,6 @@ Partition Value:    	[2010]
 Database:           	default             	 
 Table:              	alter_partition_format_test	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 #### A masked pattern was here ####
@@ -445,8 +434,6 @@ Partition Value:    	[2010]
 Database:           	default             	 
 Table:              	alter_partition_format_test	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 #### A masked pattern was here ####
@@ -496,8 +483,6 @@ Partition Value:    	[2010]
 Database:           	default             	 
 Table:              	alter_partition_format_test	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 #### A masked pattern was here ####
@@ -547,8 +532,6 @@ Partition Value:    	[2010]
 Database:           	default             	 
 Table:              	alter_partition_format_test	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 #### A masked pattern was here ####
@@ -598,8 +581,6 @@ Partition Value:    	[2010]
 Database:           	default             	 
 Table:              	alter_partition_format_test	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 #### A masked pattern was here ####

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/alter_merge_stats_orc.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/alter_merge_stats_orc.q.out b/ql/src/test/results/clientpositive/alter_merge_stats_orc.q.out
index 43cc4ef..cefe069 100644
--- a/ql/src/test/results/clientpositive/alter_merge_stats_orc.q.out
+++ b/ql/src/test/results/clientpositive/alter_merge_stats_orc.q.out
@@ -85,7 +85,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -137,7 +136,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -242,8 +240,6 @@ Partition Value:    	[2011]
 Database:           	default             	 
 Table:              	src_orc_merge_test_part_stat	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	3                   
@@ -293,8 +289,6 @@ Partition Value:    	[2011]
 Database:           	default             	 
 Table:              	src_orc_merge_test_part_stat	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	3                   
@@ -352,8 +346,6 @@ Partition Value:    	[2011]
 Database:           	default             	 
 Table:              	src_orc_merge_test_part_stat	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/alter_numbuckets_partitioned_table.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/alter_numbuckets_partitioned_table.q.out b/ql/src/test/results/clientpositive/alter_numbuckets_partitioned_table.q.out
index d3bc389..b1dfd7c 100644
--- a/ql/src/test/results/clientpositive/alter_numbuckets_partitioned_table.q.out
+++ b/ql/src/test/results/clientpositive/alter_numbuckets_partitioned_table.q.out
@@ -34,7 +34,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -82,8 +81,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	tst1                	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -145,8 +142,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	tst1                	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -184,7 +179,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -231,7 +225,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -269,8 +262,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	tst1                	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -316,7 +307,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -354,8 +344,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	tst1                	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 #### A masked pattern was here ####
@@ -406,7 +394,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -454,7 +441,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -502,7 +488,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -550,7 +535,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/alter_numbuckets_partitioned_table2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/alter_numbuckets_partitioned_table2.q.out b/ql/src/test/results/clientpositive/alter_numbuckets_partitioned_table2.q.out
index 185cf1c..e5f8e7f 100644
--- a/ql/src/test/results/clientpositive/alter_numbuckets_partitioned_table2.q.out
+++ b/ql/src/test/results/clientpositive/alter_numbuckets_partitioned_table2.q.out
@@ -32,7 +32,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -80,8 +79,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	tst1                	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -129,7 +126,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -177,8 +173,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	tst1                	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -226,7 +220,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -274,8 +267,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	tst1                	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -323,7 +314,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -371,8 +361,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	tst1                	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -420,7 +408,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -468,8 +455,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	tst1                	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -517,7 +502,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -565,8 +549,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	tst1                	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -614,7 +596,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -662,8 +643,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	tst1                	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -711,7 +690,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -759,8 +737,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	tst1                	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -808,7 +784,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -856,8 +831,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	tst1                	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/alter_numbuckets_partitioned_table2_h23.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/alter_numbuckets_partitioned_table2_h23.q.out b/ql/src/test/results/clientpositive/alter_numbuckets_partitioned_table2_h23.q.out
index 86c12c7..f919f10 100644
--- a/ql/src/test/results/clientpositive/alter_numbuckets_partitioned_table2_h23.q.out
+++ b/ql/src/test/results/clientpositive/alter_numbuckets_partitioned_table2_h23.q.out
@@ -33,7 +33,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -81,8 +80,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	tst1                	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -130,7 +127,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -178,8 +174,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	tst1                	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	8                   
@@ -227,7 +221,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -275,8 +268,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	tst1                	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	8                   
@@ -324,7 +315,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -372,8 +362,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	tst1                	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	8                   
@@ -421,7 +409,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -469,8 +456,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	tst1                	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	4                   
@@ -518,7 +503,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -566,8 +550,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	tst1                	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	4                   
@@ -615,7 +597,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -663,8 +644,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	tst1                	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	4                   
@@ -712,7 +691,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -760,8 +738,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	tst1                	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	4                   
@@ -809,7 +785,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -857,8 +832,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	tst1                	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/alter_numbuckets_partitioned_table_h23.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/alter_numbuckets_partitioned_table_h23.q.out b/ql/src/test/results/clientpositive/alter_numbuckets_partitioned_table_h23.q.out
index 42a9796..4d0f841 100644
--- a/ql/src/test/results/clientpositive/alter_numbuckets_partitioned_table_h23.q.out
+++ b/ql/src/test/results/clientpositive/alter_numbuckets_partitioned_table_h23.q.out
@@ -35,7 +35,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -83,8 +82,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	tst1                	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	8                   
@@ -146,8 +143,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	tst1                	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	12                  
@@ -185,7 +180,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -232,7 +226,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -270,8 +263,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	tst1                	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	12                  
@@ -317,7 +308,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -355,8 +345,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	tst1                	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 #### A masked pattern was here ####
@@ -407,7 +395,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -455,7 +442,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -503,7 +489,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -551,7 +536,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/alter_partition_clusterby_sortby.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/alter_partition_clusterby_sortby.q.out b/ql/src/test/results/clientpositive/alter_partition_clusterby_sortby.q.out
index eb08b6f..f5e8d1f 100644
--- a/ql/src/test/results/clientpositive/alter_partition_clusterby_sortby.q.out
+++ b/ql/src/test/results/clientpositive/alter_partition_clusterby_sortby.q.out
@@ -47,8 +47,6 @@ Partition Value:    	[abc]
 Database:           	default             	 
 Table:              	alter_table_partition_clusterby_sortby	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 #### A masked pattern was here ####
@@ -102,8 +100,6 @@ Partition Value:    	[abc]
 Database:           	default             	 
 Table:              	alter_table_partition_clusterby_sortby	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 #### A masked pattern was here ####
@@ -157,8 +153,6 @@ Partition Value:    	[abc]
 Database:           	default             	 
 Table:              	alter_table_partition_clusterby_sortby	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 #### A masked pattern was here ####
@@ -201,7 +195,6 @@ c                   	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/alter_skewed_table.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/alter_skewed_table.q.out b/ql/src/test/results/clientpositive/alter_skewed_table.q.out
index 0e0c5b0..03904e6 100644
--- a/ql/src/test/results/clientpositive/alter_skewed_table.q.out
+++ b/ql/src/test/results/clientpositive/alter_skewed_table.q.out
@@ -20,7 +20,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -59,7 +58,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -120,7 +118,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	skew_test           	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -159,7 +156,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	skew_test           	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -214,7 +210,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	skew_test           	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -255,7 +250,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	skew_test           	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/alter_table_not_sorted.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/alter_table_not_sorted.q.out b/ql/src/test/results/clientpositive/alter_table_not_sorted.q.out
index b03146b..40974e4 100644
--- a/ql/src/test/results/clientpositive/alter_table_not_sorted.q.out
+++ b/ql/src/test/results/clientpositive/alter_table_not_sorted.q.out
@@ -20,7 +20,6 @@ b                   	int
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -60,7 +59,6 @@ b                   	int
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/alter_table_serde2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/alter_table_serde2.q.out b/ql/src/test/results/clientpositive/alter_table_serde2.q.out
index dc1dae3..3b63e7d 100644
--- a/ql/src/test/results/clientpositive/alter_table_serde2.q.out
+++ b/ql/src/test/results/clientpositive/alter_table_serde2.q.out
@@ -31,7 +31,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -79,8 +78,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	tst1                	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -130,7 +127,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -179,8 +175,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	tst1                	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/alter_view_as_select.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/alter_view_as_select.q.out b/ql/src/test/results/clientpositive/alter_view_as_select.q.out
index ff2d860..c89c0dc 100644
--- a/ql/src/test/results/clientpositive/alter_view_as_select.q.out
+++ b/ql/src/test/results/clientpositive/alter_view_as_select.q.out
@@ -30,7 +30,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	tv                  	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 Table Type:         	VIRTUAL_VIEW        	 
 Table Parameters:	 	 
@@ -72,7 +71,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	tv                  	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 Table Type:         	VIRTUAL_VIEW        	 
 Table Parameters:	 	 
@@ -123,7 +121,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	tv                  	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 Table Type:         	VIRTUAL_VIEW        	 
 Table Parameters:	 	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/authorization_index.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/authorization_index.q.out b/ql/src/test/results/clientpositive/authorization_index.q.out
index 540d11b..adc02ad 100644
--- a/ql/src/test/results/clientpositive/authorization_index.q.out
+++ b/ql/src/test/results/clientpositive/authorization_index.q.out
@@ -28,7 +28,6 @@ _offsets            	array<bigint>
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	INDEX_TABLE         	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/bucket5.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/bucket5.q.out b/ql/src/test/results/clientpositive/bucket5.q.out
index 0c8418d..2e2984b 100644
--- a/ql/src/test/results/clientpositive/bucket5.q.out
+++ b/ql/src/test/results/clientpositive/bucket5.q.out
@@ -514,7 +514,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/create_alter_list_bucketing_table1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/create_alter_list_bucketing_table1.q.out b/ql/src/test/results/clientpositive/create_alter_list_bucketing_table1.q.out
index c5a253d..a5df511 100644
--- a/ql/src/test/results/clientpositive/create_alter_list_bucketing_table1.q.out
+++ b/ql/src/test/results/clientpositive/create_alter_list_bucketing_table1.q.out
@@ -31,7 +31,6 @@ col3                	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -76,7 +75,6 @@ col3                	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -126,7 +124,6 @@ col3                	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -183,7 +180,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -233,7 +229,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -282,7 +277,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -332,7 +326,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/create_like.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/create_like.q.out b/ql/src/test/results/clientpositive/create_like.q.out
index 27142c6..c93b134 100644
--- a/ql/src/test/results/clientpositive/create_like.q.out
+++ b/ql/src/test/results/clientpositive/create_like.q.out
@@ -20,7 +20,6 @@ b                   	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -59,7 +58,6 @@ b                   	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -106,7 +104,6 @@ b                   	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	EXTERNAL_TABLE      	 
@@ -338,7 +335,6 @@ last_name           	string              	last name of actor playing role
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -387,7 +383,6 @@ last_name           	string              	last name of actor playing role
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -435,7 +430,6 @@ b                   	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -475,7 +469,6 @@ col2                	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -514,7 +507,6 @@ col2                	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -563,7 +555,6 @@ col2                	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/create_like2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/create_like2.q.out b/ql/src/test/results/clientpositive/create_like2.q.out
index e5c6f9c..8b001a9 100644
--- a/ql/src/test/results/clientpositive/create_like2.q.out
+++ b/ql/src/test/results/clientpositive/create_like2.q.out
@@ -42,7 +42,6 @@ b                   	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/create_like_tbl_props.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/create_like_tbl_props.q.out b/ql/src/test/results/clientpositive/create_like_tbl_props.q.out
index e151897..d7f9dd2 100644
--- a/ql/src/test/results/clientpositive/create_like_tbl_props.q.out
+++ b/ql/src/test/results/clientpositive/create_like_tbl_props.q.out
@@ -24,7 +24,6 @@ value               	string              	default
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -68,7 +67,6 @@ value               	string              	default
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -112,7 +110,6 @@ value               	string              	default
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -157,7 +154,6 @@ value               	string              	default
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -212,7 +208,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/create_like_view.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/create_like_view.q.out b/ql/src/test/results/clientpositive/create_like_view.q.out
index 0978fcf..e2dc2c4 100644
--- a/ql/src/test/results/clientpositive/create_like_view.q.out
+++ b/ql/src/test/results/clientpositive/create_like_view.q.out
@@ -48,7 +48,6 @@ b                   	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -105,7 +104,6 @@ b                   	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -160,7 +158,6 @@ b                   	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	EXTERNAL_TABLE      	 
@@ -281,7 +278,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/create_or_replace_view.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/create_or_replace_view.q.out b/ql/src/test/results/clientpositive/create_or_replace_view.q.out
index aab78ba..dd5bf13 100644
--- a/ql/src/test/results/clientpositive/create_or_replace_view.q.out
+++ b/ql/src/test/results/clientpositive/create_or_replace_view.q.out
@@ -30,7 +30,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	vt                  	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 Table Type:         	VIRTUAL_VIEW        	 
 Table Parameters:	 	 
@@ -121,7 +120,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	vt                  	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 Table Type:         	VIRTUAL_VIEW        	 
 Table Parameters:	 	 
@@ -215,7 +213,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	vt                  	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 Table Type:         	VIRTUAL_VIEW        	 
 Table Parameters:	 	 
@@ -286,7 +283,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	vt                  	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 Table Type:         	VIRTUAL_VIEW        	 
 Table Parameters:	 	 
@@ -378,7 +374,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	vt                  	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 Table Type:         	VIRTUAL_VIEW        	 
 Table Parameters:	 	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/create_skewed_table1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/create_skewed_table1.q.out b/ql/src/test/results/clientpositive/create_skewed_table1.q.out
index e8d85a5..415bb77 100644
--- a/ql/src/test/results/clientpositive/create_skewed_table1.q.out
+++ b/ql/src/test/results/clientpositive/create_skewed_table1.q.out
@@ -36,7 +36,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -69,7 +68,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -103,7 +101,6 @@ col3                	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/create_view.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/create_view.q.out b/ql/src/test/results/clientpositive/create_view.q.out
index 67e87b9..1038d01 100644
--- a/ql/src/test/results/clientpositive/create_view.q.out
+++ b/ql/src/test/results/clientpositive/create_view.q.out
@@ -250,7 +250,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 Table Type:         	VIRTUAL_VIEW        	 
 Table Parameters:	 	 
@@ -300,7 +299,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 Table Type:         	VIRTUAL_VIEW        	 
 Table Parameters:	 	 
@@ -347,7 +345,6 @@ valoo               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 Table Type:         	VIRTUAL_VIEW        	 
 Table Parameters:	 	 
@@ -396,7 +393,6 @@ valoo               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 Table Type:         	VIRTUAL_VIEW        	 
 Table Parameters:	 	 
@@ -761,7 +757,6 @@ c                   	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 Table Type:         	VIRTUAL_VIEW        	 
 Table Parameters:	 	 
@@ -842,7 +837,6 @@ m                   	int
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 Table Type:         	VIRTUAL_VIEW        	 
 Table Parameters:	 	 
@@ -921,7 +915,6 @@ m                   	int
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 Table Type:         	VIRTUAL_VIEW        	 
 Table Parameters:	 	 
@@ -990,7 +983,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 Table Type:         	VIRTUAL_VIEW        	 
 Table Parameters:	 	 
@@ -1067,7 +1059,6 @@ boom                	int
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 Table Type:         	VIRTUAL_VIEW        	 
 Table Parameters:	 	 
@@ -1140,7 +1131,6 @@ mycol               	int
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 Table Type:         	VIRTUAL_VIEW        	 
 Table Parameters:	 	 
@@ -1226,7 +1216,6 @@ key                 	int
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 Table Type:         	VIRTUAL_VIEW        	 
 Table Parameters:	 	 
@@ -1348,7 +1337,6 @@ v2                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 Table Type:         	VIRTUAL_VIEW        	 
 Table Parameters:	 	 
@@ -1465,7 +1453,6 @@ value_count         	bigint
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 Table Type:         	VIRTUAL_VIEW        	 
 Table Parameters:	 	 
@@ -1551,7 +1538,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 Table Type:         	VIRTUAL_VIEW        	 
 Table Parameters:	 	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/create_view_partitioned.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/create_view_partitioned.q.out b/ql/src/test/results/clientpositive/create_view_partitioned.q.out
index ebf9a6b..caa2251 100644
--- a/ql/src/test/results/clientpositive/create_view_partitioned.q.out
+++ b/ql/src/test/results/clientpositive/create_view_partitioned.q.out
@@ -71,7 +71,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 Table Type:         	VIRTUAL_VIEW        	 
 Table Parameters:	 	 
@@ -279,7 +278,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 Table Type:         	VIRTUAL_VIEW        	 
 Table Parameters:	 	 
@@ -393,7 +391,6 @@ v                   	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 Table Type:         	VIRTUAL_VIEW        	 
 Table Parameters:	 	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/create_view_translate.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/create_view_translate.q.out b/ql/src/test/results/clientpositive/create_view_translate.q.out
index fd74058..886a01b 100644
--- a/ql/src/test/results/clientpositive/create_view_translate.q.out
+++ b/ql/src/test/results/clientpositive/create_view_translate.q.out
@@ -29,7 +29,6 @@ key                 	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 Table Type:         	VIRTUAL_VIEW        	 
 Table Parameters:	 	 
@@ -75,7 +74,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 Table Type:         	VIRTUAL_VIEW        	 
 Table Parameters:	 	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/ctas.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/ctas.q.out b/ql/src/test/results/clientpositive/ctas.q.out
index 6f17a74..24cabc5 100644
--- a/ql/src/test/results/clientpositive/ctas.q.out
+++ b/ql/src/test/results/clientpositive/ctas.q.out
@@ -146,7 +146,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -294,7 +293,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -442,7 +440,6 @@ conb                	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -507,7 +504,6 @@ conb                	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -656,7 +652,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/ctas_colname.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/ctas_colname.q.out b/ql/src/test/results/clientpositive/ctas_colname.q.out
index 3ecfbf2..232d505 100644
--- a/ql/src/test/results/clientpositive/ctas_colname.q.out
+++ b/ql/src/test/results/clientpositive/ctas_colname.q.out
@@ -103,7 +103,6 @@ c2                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -261,7 +260,6 @@ rr                  	int
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -449,7 +447,6 @@ lead1               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -620,7 +617,6 @@ _c1                 	double
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -769,7 +765,6 @@ _c1                 	bigint
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -1205,7 +1200,6 @@ _c1                 	bigint
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -1343,7 +1337,6 @@ key                 	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/ctas_hadoop20.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/ctas_hadoop20.q.out b/ql/src/test/results/clientpositive/ctas_hadoop20.q.out
index b76028b..4af1e0a 100644
--- a/ql/src/test/results/clientpositive/ctas_hadoop20.q.out
+++ b/ql/src/test/results/clientpositive/ctas_hadoop20.q.out
@@ -139,7 +139,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -283,7 +282,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -428,7 +426,6 @@ conb                	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -493,7 +490,6 @@ conb                	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -638,7 +634,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/ctas_uses_database_location.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/ctas_uses_database_location.q.out b/ql/src/test/results/clientpositive/ctas_uses_database_location.q.out
index 64d9cb3..554ae48 100644
--- a/ql/src/test/results/clientpositive/ctas_uses_database_location.q.out
+++ b/ql/src/test/results/clientpositive/ctas_uses_database_location.q.out
@@ -138,7 +138,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	db1                 	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/database_location.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/database_location.q.out b/ql/src/test/results/clientpositive/database_location.q.out
index 159a8e2..797177d 100644
--- a/ql/src/test/results/clientpositive/database_location.q.out
+++ b/ql/src/test/results/clientpositive/database_location.q.out
@@ -39,7 +39,6 @@ value               	int
 # Detailed Table Information	 	 
 Database:           	db1                 	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -110,7 +109,6 @@ value               	int
 # Detailed Table Information	 	 
 Database:           	db2                 	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/decimal_serde.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/decimal_serde.q.out b/ql/src/test/results/clientpositive/decimal_serde.q.out
index d651799..0783d9a 100644
--- a/ql/src/test/results/clientpositive/decimal_serde.q.out
+++ b/ql/src/test/results/clientpositive/decimal_serde.q.out
@@ -110,7 +110,6 @@ value               	int
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -162,7 +161,6 @@ value               	int
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/default_file_format.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/default_file_format.q.out b/ql/src/test/results/clientpositive/default_file_format.q.out
index 1ffba08..3d5c20f 100644
--- a/ql/src/test/results/clientpositive/default_file_format.q.out
+++ b/ql/src/test/results/clientpositive/default_file_format.q.out
@@ -55,7 +55,6 @@ c                   	int
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -85,7 +84,6 @@ c                   	int
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -115,7 +113,6 @@ c                   	int
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -145,7 +142,6 @@ c                   	int
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	EXTERNAL_TABLE      	 
@@ -176,7 +172,6 @@ c                   	int
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/describe_comment_indent.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/describe_comment_indent.q.out b/ql/src/test/results/clientpositive/describe_comment_indent.q.out
index 5ded495..3e0f45e 100644
--- a/ql/src/test/results/clientpositive/describe_comment_indent.q.out
+++ b/ql/src/test/results/clientpositive/describe_comment_indent.q.out
@@ -56,7 +56,6 @@ col3                	string              	col3
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/describe_comment_nonascii.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/describe_comment_nonascii.q.out b/ql/src/test/results/clientpositive/describe_comment_nonascii.q.out
index 12f385d..b202e65 100644
--- a/ql/src/test/results/clientpositive/describe_comment_nonascii.q.out
+++ b/ql/src/test/results/clientpositive/describe_comment_nonascii.q.out
@@ -49,7 +49,6 @@ col3                	string              	わご_col3
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/describe_formatted_view_partitioned.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/describe_formatted_view_partitioned.q.out b/ql/src/test/results/clientpositive/describe_formatted_view_partitioned.q.out
index 18768d9..2c8b0b0 100644
--- a/ql/src/test/results/clientpositive/describe_formatted_view_partitioned.q.out
+++ b/ql/src/test/results/clientpositive/describe_formatted_view_partitioned.q.out
@@ -55,8 +55,6 @@ Partition Value:    	[val_86]
 Database:           	default             	 
 Table:              	view_partitioned    	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 #### A masked pattern was here ####
 PREHOOK: query: DROP VIEW view_partitioned


[30/50] [abbrv] hive git commit: HIVE-11230 IMetaStoreClient getPartition method arguments incorrectly named (Elliot West via gates)

Posted by ga...@apache.org.
HIVE-11230 IMetaStoreClient getPartition method arguments incorrectly named (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/b0247cff
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/b0247cff
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/b0247cff

Branch: refs/heads/hbase-metastore
Commit: b0247cffcd6f16de8923d14580c980bc9a6c0980
Parents: 7fe23aa
Author: Alan Gates <ga...@hortonworks.com>
Authored: Fri Jul 17 13:01:57 2015 -0700
Committer: Alan Gates <ga...@hortonworks.com>
Committed: Fri Jul 17 13:01:57 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/hive/blob/b0247cff/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java b/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
index 147ffcc..9bf4bfb 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
@@ -417,8 +417,8 @@ public interface IMetaStoreClient {
       throws InvalidObjectException, AlreadyExistsException, MetaException, TException;
 
   /**
-   * @param tblName
    * @param dbName
+   * @param tblName
    * @param partVals
    * @return the partition object
    * @throws MetaException
@@ -426,7 +426,7 @@ public interface IMetaStoreClient {
    * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#get_partition(java.lang.String,
    *      java.lang.String, java.util.List)
    */
-  Partition getPartition(String tblName, String dbName,
+  Partition getPartition(String dbName, String tblName,
       List<String> partVals) throws NoSuchObjectException, MetaException, TException;
 
   /**


[11/50] [abbrv] hive git commit: HIVE-11124. Move OrcRecordUpdater.getAcidEventFields to RecordReaderFactory.

Posted by ga...@apache.org.
HIVE-11124. Move OrcRecordUpdater.getAcidEventFields to RecordReaderFactory.


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

Branch: refs/heads/hbase-metastore
Commit: 240097b78b70172e1cf9bc37876a566ddfb9e115
Parents: 4232695
Author: Owen O'Malley <om...@apache.org>
Authored: Thu Jun 25 22:02:47 2015 -0700
Committer: Owen O'Malley <om...@apache.org>
Committed: Wed Jul 15 14:41:42 2015 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/ql/io/orc/OrcRecordUpdater.java    | 5 -----
 .../org/apache/hadoop/hive/ql/io/orc/RecordReaderFactory.java | 7 ++++++-
 2 files changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/240097b7/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRecordUpdater.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRecordUpdater.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRecordUpdater.java
index e4651b8..2220b8e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRecordUpdater.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcRecordUpdater.java
@@ -207,11 +207,6 @@ public class OrcRecordUpdater implements RecordUpdater {
     return new OrcStruct.OrcStructInspector(fields);
   }
 
-  public static List<String> getAcidEventFields() {
-    return Lists.newArrayList("operation", "originalTransaction", "bucket", "rowId",
-        "currentTransaction", "row");
-  }
-
   OrcRecordUpdater(Path path,
                    AcidOutputFormat.Options options) throws IOException {
     this.options = options;

http://git-wip-us.apache.org/repos/asf/hive/blob/240097b7/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderFactory.java
index 8740ee6..23a9af4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderFactory.java
@@ -72,9 +72,14 @@ public class RecordReaderFactory {
     }
   }
 
+  static List<String> getAcidEventFields() {
+    return Lists.newArrayList("operation", "originalTransaction", "bucket",
+        "rowId", "currentTransaction", "row");
+  }
+
   private static boolean checkAcidSchema(List<OrcProto.Type> fileSchema) {
     if (fileSchema.get(0).getKind().equals(OrcProto.Type.Kind.STRUCT)) {
-      List<String> acidFields = OrcRecordUpdater.getAcidEventFields();
+      List<String> acidFields = getAcidEventFields();
       List<String> rootFields = fileSchema.get(0).getFieldNamesList();
       if (acidFields.equals(rootFields)) {
         return true;


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

Posted by ga...@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/hbase-metastore
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


[23/50] [abbrv] hive git commit: HIVE-11216: UDF GenericUDFMapKeys throws NPE when a null map value is passed in (Yibing Shi via Chaoyu Tang, reviewed by Szehon Ho

Posted by ga...@apache.org.
HIVE-11216: UDF GenericUDFMapKeys throws NPE when a null map value is passed in (Yibing Shi via Chaoyu Tang, reviewed by Szehon Ho


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

Branch: refs/heads/hbase-metastore
Commit: 854950b2a0ce5b3885b2be0e7359f04b483f687c
Parents: e74dc32
Author: ctang <ct...@gmail.com>
Authored: Thu Jul 16 14:34:28 2015 -0400
Committer: ctang <ct...@gmail.com>
Committed: Thu Jul 16 14:34:28 2015 -0400

----------------------------------------------------------------------
 .../apache/hadoop/hive/ql/udf/generic/GenericUDFMapKeys.java   | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/854950b2/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFMapKeys.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFMapKeys.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFMapKeys.java
index d8afd74..d0cff08 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFMapKeys.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFMapKeys.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hive.ql.udf.generic;
 
 import java.util.ArrayList;
+import java.util.Map;
 
 import org.apache.hadoop.hive.ql.exec.Description;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
@@ -61,7 +62,10 @@ public class GenericUDFMapKeys extends GenericUDF {
   public Object evaluate(DeferredObject[] arguments) throws HiveException {
     retArray.clear();
     Object mapObj = arguments[0].get();
-    retArray.addAll(mapOI.getMap(mapObj).keySet());
+    Map<?,?> mapVal = mapOI.getMap(mapObj);
+    if (mapVal != null) {
+      retArray.addAll(mapVal.keySet());
+    }
     return retArray;
   }
 


[12/50] [abbrv] hive git commit: HIVE-11158 Add tests for HPL/SQL (Dmitry Tolpeko via gates)

Posted by ga...@apache.org.
HIVE-11158 Add tests for HPL/SQL (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/7338d8e1
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/7338d8e1
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/7338d8e1

Branch: refs/heads/hbase-metastore
Commit: 7338d8e11983bfe7a63aadfd82b64adef765cb67
Parents: 240097b
Author: Alan Gates <ga...@hortonworks.com>
Authored: Wed Jul 15 17:09:12 2015 -0700
Committer: Alan Gates <ga...@hortonworks.com>
Committed: Wed Jul 15 17:09:12 2015 -0700

----------------------------------------------------------------------
 hplsql/pom.xml                                  |   9 +-
 .../main/java/org/apache/hive/hplsql/Exec.java  |  11 +-
 .../org/apache/hive/hplsql/TestHplsqlLocal.java | 330 +++++++++++++++++++
 hplsql/src/test/queries/local/add.sql           |   2 +
 hplsql/src/test/queries/local/assign.sql        |   7 +
 hplsql/src/test/queries/local/bool_expr.sql     |  47 +++
 hplsql/src/test/queries/local/break.sql         |  10 +
 hplsql/src/test/queries/local/case.sql          |  35 ++
 hplsql/src/test/queries/local/cast.sql          |   4 +
 hplsql/src/test/queries/local/char.sql          |   1 +
 hplsql/src/test/queries/local/coalesce.sql      |   4 +
 hplsql/src/test/queries/local/concat.sql        |   2 +
 .../src/test/queries/local/create_function.sql  |  11 +
 .../src/test/queries/local/create_function2.sql |  11 +
 .../src/test/queries/local/create_procedure.sql |   9 +
 hplsql/src/test/queries/local/date.sql          |   5 +
 hplsql/src/test/queries/local/dbms_output.sql   |   6 +
 hplsql/src/test/queries/local/declare.sql       |  16 +
 .../test/queries/local/declare_condition.sql    |   8 +
 .../test/queries/local/declare_condition2.sql   |  10 +
 hplsql/src/test/queries/local/decode.sql        |  10 +
 hplsql/src/test/queries/local/equal.sql         |  55 ++++
 hplsql/src/test/queries/local/exception.sql     |  14 +
 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 +
 hplsql/src/test/queries/local/exit.sql          |  31 ++
 hplsql/src/test/queries/local/expr.sql          |  21 ++
 hplsql/src/test/queries/local/for_range.sql     |  20 ++
 hplsql/src/test/queries/local/if.sql            |  68 ++++
 hplsql/src/test/queries/local/instr.sql         |  49 +++
 hplsql/src/test/queries/local/interval.sql      |  15 +
 hplsql/src/test/queries/local/lang.sql          |  57 ++++
 hplsql/src/test/queries/local/leave.sql         |  33 ++
 hplsql/src/test/queries/local/len.sql           |   1 +
 hplsql/src/test/queries/local/length.sql        |   1 +
 hplsql/src/test/queries/local/lower.sql         |   1 +
 hplsql/src/test/queries/local/nvl.sql           |   4 +
 hplsql/src/test/queries/local/nvl2.sql          |   2 +
 hplsql/src/test/queries/local/print.sql         |   5 +
 hplsql/src/test/queries/local/return.sql        |   3 +
 hplsql/src/test/queries/local/seterror.sql      |  10 +
 hplsql/src/test/queries/local/sub.sql           |   1 +
 hplsql/src/test/queries/local/substr.sql        |   2 +
 hplsql/src/test/queries/local/substring.sql     |   8 +
 hplsql/src/test/queries/local/timestamp.sql     |   4 +
 hplsql/src/test/queries/local/timestamp_iso.sql |   2 +
 hplsql/src/test/queries/local/to_char.sql       |   1 +
 hplsql/src/test/queries/local/to_timestamp.sql  |   5 +
 hplsql/src/test/queries/local/trim.sql          |   1 +
 hplsql/src/test/queries/local/twopipes.sql      |   1 +
 hplsql/src/test/queries/local/upper.sql         |   1 +
 hplsql/src/test/queries/local/values_into.sql   |   6 +
 hplsql/src/test/queries/local/while.sql         |  20 ++
 hplsql/src/test/results/local/add.out.txt       |   2 +
 hplsql/src/test/results/local/assign.out.txt    |   8 +
 hplsql/src/test/results/local/bool_expr.out.txt |  32 ++
 hplsql/src/test/results/local/break.out.txt     |  29 ++
 hplsql/src/test/results/local/case.out.txt      |  12 +
 hplsql/src/test/results/local/cast.out.txt      |   8 +
 hplsql/src/test/results/local/char.out.txt      |   1 +
 hplsql/src/test/results/local/coalesce.out.txt  |   4 +
 hplsql/src/test/results/local/concat.out.txt    |   2 +
 .../test/results/local/create_function.out.txt  |   9 +
 .../test/results/local/create_function2.out.txt |  10 +
 .../test/results/local/create_procedure.out.txt |   8 +
 hplsql/src/test/results/local/date.out.txt      |   4 +
 .../src/test/results/local/dbms_output.out.txt  |   3 +
 hplsql/src/test/results/local/declare.out.txt   |  13 +
 .../results/local/declare_condition.out.txt     |   7 +
 .../results/local/declare_condition2.out.txt    |  12 +
 hplsql/src/test/results/local/decode.out.txt    |  13 +
 hplsql/src/test/results/local/equal.out.txt     |  48 +++
 hplsql/src/test/results/local/exception.out.txt |  13 +
 .../src/test/results/local/exception2.out.txt   |   5 +
 hplsql/src/test/results/local/exit.out.txt      |  42 +++
 hplsql/src/test/results/local/expr.out.txt      |  29 ++
 hplsql/src/test/results/local/for_range.out.txt |  65 ++++
 hplsql/src/test/results/local/if.out.txt        |  40 +++
 hplsql/src/test/results/local/instr.out.txt     |  33 ++
 hplsql/src/test/results/local/interval.out.txt  |  11 +
 hplsql/src/test/results/local/lang.out.txt      |  34 ++
 hplsql/src/test/results/local/leave.out.txt     |  42 +++
 hplsql/src/test/results/local/len.out.txt       |   1 +
 hplsql/src/test/results/local/length.out.txt    |   1 +
 hplsql/src/test/results/local/lower.out.txt     |   1 +
 hplsql/src/test/results/local/nvl.out.txt       |   4 +
 hplsql/src/test/results/local/nvl2.out.txt      |   2 +
 .../test/results/local/plhqlexception.out.txt   |   6 +
 .../test/results/local/plhqlexception1.out.txt  |  10 +
 .../test/results/local/plhqlexception2.out.txt  | 106 ++++++
 hplsql/src/test/results/local/print.out.txt     |   6 +
 hplsql/src/test/results/local/return.out.txt    |   3 +
 .../results/local/select_conversion.out.txt     |   9 +
 hplsql/src/test/results/local/seterror.out.txt  |   6 +
 hplsql/src/test/results/local/sub.out.txt       |   1 +
 hplsql/src/test/results/local/substr.out.txt    |   2 +
 hplsql/src/test/results/local/substring.out.txt |   8 +
 hplsql/src/test/results/local/timestamp.out.txt |   4 +
 .../test/results/local/timestamp_iso.out.txt    |   2 +
 hplsql/src/test/results/local/to_char.out.txt   |   1 +
 .../src/test/results/local/to_timestamp.out.txt |   4 +
 hplsql/src/test/results/local/trim.out.txt      |   1 +
 hplsql/src/test/results/local/twopipes.out.txt  |   1 +
 hplsql/src/test/results/local/upper.out.txt     |   1 +
 .../src/test/results/local/values_into.out.txt  |  11 +
 hplsql/src/test/results/local/while.out.txt     |  72 ++++
 108 files changed, 1821 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/pom.xml
----------------------------------------------------------------------
diff --git a/hplsql/pom.xml b/hplsql/pom.xml
index d096e90..fc1c527 100644
--- a/hplsql/pom.xml
+++ b/hplsql/pom.xml
@@ -58,7 +58,6 @@
       <artifactId>commons-logging</artifactId>
       <version>${commons-logging.version}</version>
     </dependency>
-
     <dependency>
       <groupId>commons-io</groupId>
       <artifactId>commons-io</artifactId>
@@ -74,6 +73,12 @@
        <artifactId>antlr4-runtime</artifactId>
        <version>4.5</version>
     </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <version>${junit.version}</version>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <profiles>
@@ -103,7 +108,7 @@
   
   <build>
     <plugins>
-      <plugin>
+     <plugin>
         <groupId>org.antlr</groupId>
           <artifactId>antlr4-maven-plugin</artifactId>
           <version>4.5</version>

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/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 9ec8959..40fdc82 100644
--- a/hplsql/src/main/java/org/apache/hive/hplsql/Exec.java
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Exec.java
@@ -1644,7 +1644,16 @@ public class Exec extends HplsqlBaseVisitor<Integer> {
    */
   @Override 
   public Integer visitLabel(HplsqlParser.LabelContext ctx) { 
-    exec.labels.push(ctx.L_ID().toString()); 
+    if (ctx.L_ID() != null) {
+      exec.labels.push(ctx.L_ID().toString());
+    }
+    else {
+      String label = ctx.L_LABEL().getText();
+      if (label.endsWith(":")) {
+        label = label.substring(0, label.length() - 1);
+      }
+      exec.labels.push(label);
+    }
     return 0;
   }
   

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/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
new file mode 100644
index 0000000..ee2be66
--- /dev/null
+++ b/hplsql/src/test/java/org/apache/hive/hplsql/TestHplsqlLocal.java
@@ -0,0 +1,330 @@
+/**
+ * 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.hive.hplsql;
+
+import java.io.BufferedReader;
+import java.io.ByteArrayOutputStream;
+import java.io.PrintStream;
+import java.io.StringReader;
+import org.apache.commons.io.FileUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Unit tests for HPL/SQL (no Hive connection required)
+ */
+public class TestHplsqlLocal {
+
+  private final ByteArrayOutputStream out = new ByteArrayOutputStream();
+
+  @Test
+  public void testAdd() throws Exception {
+    run("add");
+  }
+
+  @Test
+  public void testAssign() throws Exception {
+    run("assign");
+  }
+
+  @Test
+  public void testBoolExpr() throws Exception {
+    run("bool_expr");
+  }
+
+  @Test
+  public void testBreak() throws Exception {
+    run("break");
+  }
+
+  @Test
+  public void testCase() throws Exception {
+    run("case");
+  }
+
+  @Test
+  public void testCast() throws Exception {
+    run("cast");
+  }
+
+  @Test
+  public void testChar() throws Exception {
+    run("char");
+  }
+
+  @Test
+  public void testCoalesce() throws Exception {
+    run("coalesce");
+  }
+
+  @Test
+  public void testConcat() throws Exception {
+    run("concat");
+  }
+
+  @Test
+  public void testCreateFunction() throws Exception {
+    run("create_function");
+  }
+
+  @Test
+  public void testCreateFunction2() throws Exception {
+    run("create_function2");
+  }
+
+  @Test
+  public void testCreateProcedure() throws Exception {
+    run("create_procedure");
+  }
+
+  @Test
+  public void testDate() throws Exception {
+    run("date");
+  }
+
+  @Test
+  public void testDbmsOutput() throws Exception {
+    run("dbms_output");
+  }
+
+  @Test
+  public void testDeclare() throws Exception {
+    run("declare");
+  }
+
+  @Test
+  public void testDeclareCondition() throws Exception {
+    run("declare_condition");
+  }
+
+  @Test
+  public void testDeclareCondition2() throws Exception {
+    run("declare_condition2");
+  }
+
+  @Test
+  public void testDecode() throws Exception {
+    run("decode");
+  }
+
+  @Test
+  public void testEqual() throws Exception {
+    run("equal");
+  }
+
+  @Test
+  public void testException() throws Exception {
+    run("exception");
+  }
+
+  @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");
+  }
+
+  @Test
+  public void testExpr() throws Exception {
+    run("expr");
+  }
+
+  @Test
+  public void testForRange() throws Exception {
+    run("for_range");
+  }
+
+  @Test
+  public void testIf() throws Exception {
+    run("if");
+  }
+
+  @Test
+  public void testInstr() throws Exception {
+    run("instr");
+  }
+
+  @Test
+  public void testInterval() throws Exception {
+    run("interval");
+  }
+
+  @Test
+  public void testLang() throws Exception {
+    run("lang");
+  }
+
+  @Test
+  public void testLeave() throws Exception {
+    run("leave");
+  }
+
+  @Test
+  public void testLength() throws Exception {
+    run("length");
+  }
+
+  @Test
+  public void testLen() throws Exception {
+    run("len");
+  }
+
+  @Test
+  public void testLower() throws Exception {
+    run("lower");
+  }
+
+  @Test
+  public void testNvl() throws Exception {
+    run("nvl");
+  }
+
+  @Test
+  public void testNvl2() throws Exception {
+    run("nvl2");
+  }
+
+  @Test
+  public void testPrint() throws Exception {
+    run("print");
+  }
+
+  @Test
+  public void testReturn() throws Exception {
+    run("return");
+  }
+
+  @Test
+  public void testSetError() throws Exception {
+    run("seterror");
+  }
+
+  @Test
+  public void testSub() throws Exception {
+    run("sub");
+  }
+
+  @Test
+  public void testSubstring() throws Exception {
+    run("substring");
+  }
+
+  @Test
+  public void testSubstr() throws Exception {
+    run("substr");
+  }
+
+  @Test
+  public void testTimestampIso() throws Exception {
+    run("timestamp_iso");
+  }
+
+  @Test
+  public void testTimestamp() throws Exception {
+    run("timestamp");
+  }
+
+  @Test
+  public void testToChar() throws Exception {
+    run("to_char");
+  }
+
+  @Test
+  public void testToTimestamp() throws Exception {
+    run("to_timestamp");
+  }
+
+  @Test
+  public void testTrim() throws Exception {
+    run("trim");
+  }
+
+  @Test
+  public void testTwoPipes() throws Exception {
+    run("twopipes");
+  }
+
+  @Test
+  public void testUpper() throws Exception {
+    run("upper");
+  }
+
+  @Test
+  public void testValuesInto() throws Exception {
+    run("values_into");
+  }
+
+  @Test
+  public void testWhile() throws Exception {
+    run("while");
+  }
+
+  /**
+   * Run a test file
+   */
+  void run(String testFile) throws Exception {
+    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());
+    }
+    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();
+    System.setOut(null);
+    Assert.assertEquals(s, t);
+  }
+
+  /**
+   * Get test output
+   */
+  String getTestOutput(String s) throws Exception {
+    StringBuilder sb = new StringBuilder();
+    BufferedReader reader = new BufferedReader(new StringReader(s));
+    String line = null;
+    while ((line = reader.readLine()) != null) {
+      if (!line.startsWith("log4j:")) {
+        sb.append(line);
+        sb.append("\n");
+      }
+    }
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/add.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/add.sql b/hplsql/src/test/queries/local/add.sql
new file mode 100644
index 0000000..2861a5f
--- /dev/null
+++ b/hplsql/src/test/queries/local/add.sql
@@ -0,0 +1,2 @@
+DATE '2014-12-31' + 1;
+1 + DATE '2014-12-31';
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/assign.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/assign.sql b/hplsql/src/test/queries/local/assign.sql
new file mode 100644
index 0000000..67e6893
--- /dev/null
+++ b/hplsql/src/test/queries/local/assign.sql
@@ -0,0 +1,7 @@
+code := 'A';
+status := 1;
+count = 0;
+
+SET code = 'A';
+SET status = 1, count = 0;
+SET (count, limit) = (0, 100);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/bool_expr.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/bool_expr.sql b/hplsql/src/test/queries/local/bool_expr.sql
new file mode 100644
index 0000000..098096f
--- /dev/null
+++ b/hplsql/src/test/queries/local/bool_expr.sql
@@ -0,0 +1,47 @@
+IF 1=1 THEN
+  PRINT 'Correct';
+ELSE 
+  PRINT 'FAILED';
+END IF;
+
+IF 1=1 OR 2=2 THEN
+  PRINT 'Correct';
+ELSE 
+  PRINT 'FAILED';
+END IF;
+
+IF (1=1 OR 2=2) THEN
+  PRINT 'Correct';
+ELSE 
+  PRINT 'FAILED';
+END IF; 
+
+IF (1=1 AND 2=2 AND 3=4) THEN
+  PRINT 'FAILED';
+ELSE 
+  PRINT 'Correct';
+END IF; 
+
+IF ((1=1) AND (2=2)) THEN
+  PRINT 'Correct';
+ELSE 
+  PRINT 'FAILED';
+END IF;
+
+IF (1=1 AND (2=2)) THEN
+  PRINT 'Correct';
+ELSE 
+  PRINT 'FAILED';
+END IF;
+
+IF ((1=1) AND 2=2 AND 3=3) THEN
+  PRINT 'Correct';
+ELSE 
+  PRINT 'FAILED';
+END IF;
+
+IF ((1=1 OR 2=2) AND 2=2 AND 3=3 AND (1=2 OR 2=3)) THEN
+  PRINT 'FAILED';
+ELSE 
+  PRINT 'Correct';
+END IF;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/break.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/break.sql b/hplsql/src/test/queries/local/break.sql
new file mode 100644
index 0000000..c53535d
--- /dev/null
+++ b/hplsql/src/test/queries/local/break.sql
@@ -0,0 +1,10 @@
+DECLARE count INT DEFAULT 3;
+WHILE 1=1 BEGIN
+  PRINT 'Start of while block';
+  PRINT count;
+  SET count = count - 1;
+  IF count = 0
+    BREAK;
+  PRINT 'End of while block';
+END
+PRINT 'End of script';
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/case.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/case.sql b/hplsql/src/test/queries/local/case.sql
new file mode 100644
index 0000000..5bbdda9
--- /dev/null
+++ b/hplsql/src/test/queries/local/case.sql
@@ -0,0 +1,35 @@
+PRINT CASE 1 
+        WHEN 0 THEN 'FAILED'
+        WHEN 1 THEN 'Correct' 
+        WHEN 2 THEN 'FAILED'
+        ELSE 'FAILED'
+      END 
+
+PRINT CASE 3 
+        WHEN 0 THEN 'FAILED'
+        WHEN 1 THEN 'FAILED'
+        ELSE 'Correct'
+      END       
+      
+PRINT NVL2(CASE 3 
+        WHEN 0 THEN 'FAILED'
+        WHEN 1 THEN 'FAILED'
+      END, 'FAILED', 'Correct')  
+      
+PRINT CASE  
+        WHEN 1=0 THEN 'FAILED'
+        WHEN 1=1 THEN 'Correct' 
+        WHEN 1=2 THEN 'FAILED'
+        ELSE 'FAILED'
+      END 
+
+PRINT CASE  
+        WHEN 3=0 THEN 'FAILED'
+        WHEN 3=1 THEN 'FAILED'
+        ELSE 'Correct'
+      END       
+      
+PRINT NVL2(CASE  
+        WHEN 3=0 THEN 'FAILED'
+        WHEN 3=1 THEN 'FAILED'
+      END, 'FAILED', 'Correct') 
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/cast.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/cast.sql b/hplsql/src/test/queries/local/cast.sql
new file mode 100644
index 0000000..3adab22
--- /dev/null
+++ b/hplsql/src/test/queries/local/cast.sql
@@ -0,0 +1,4 @@
+CAST('Abc' AS CHAR(1));
+CAST('Abc' AS VARCHAR(2));
+CAST('Abc' AS CHAR);
+CAST(TIMESTAMP '2015-03-12 10:58:34.111' AS CHAR(10))

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/char.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/char.sql b/hplsql/src/test/queries/local/char.sql
new file mode 100644
index 0000000..2a4f779
--- /dev/null
+++ b/hplsql/src/test/queries/local/char.sql
@@ -0,0 +1 @@
+CHAR(1000)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/coalesce.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/coalesce.sql b/hplsql/src/test/queries/local/coalesce.sql
new file mode 100644
index 0000000..4b65d58
--- /dev/null
+++ b/hplsql/src/test/queries/local/coalesce.sql
@@ -0,0 +1,4 @@
+COALESCE('First non-null', 1);
+COALESCE(NULL, 'First non-null');
+COALESCE(NULL, 'First non-null', 1);
+COALESCE(NULL, NULL, 'First non-null', 1);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/concat.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/concat.sql b/hplsql/src/test/queries/local/concat.sql
new file mode 100644
index 0000000..b7769bb
--- /dev/null
+++ b/hplsql/src/test/queries/local/concat.sql
@@ -0,0 +1,2 @@
+CONCAT('a', 'b', NULL, 'c'); 
+NVL(CONCAT(NULL, NULL, NULL), 'NULL Value'); 
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/create_function.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/create_function.sql b/hplsql/src/test/queries/local/create_function.sql
new file mode 100644
index 0000000..96bf290
--- /dev/null
+++ b/hplsql/src/test/queries/local/create_function.sql
@@ -0,0 +1,11 @@
+CREATE FUNCTION hello()
+ RETURNS STRING
+BEGIN
+ PRINT 'Start';
+ RETURN 'Hello, world';
+ PRINT 'Must not be printed';
+END;
+ 
+-- Call the function
+PRINT hello() || '!';
+PRINT 'End of script';
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/create_function2.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/create_function2.sql b/hplsql/src/test/queries/local/create_function2.sql
new file mode 100644
index 0000000..744ea9e
--- /dev/null
+++ b/hplsql/src/test/queries/local/create_function2.sql
@@ -0,0 +1,11 @@
+CREATE FUNCTION hello2(text STRING)
+  RETURNS STRING
+BEGIN
+  PRINT 'Start';
+  RETURN 'Hello, ' || text || '!';
+  PRINT 'Must not be printed';
+END;
+ 
+-- Call the function
+PRINT hello2('wor' || 'ld');
+PRINT 'End of script';
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/create_procedure.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/create_procedure.sql b/hplsql/src/test/queries/local/create_procedure.sql
new file mode 100644
index 0000000..28088a2
--- /dev/null
+++ b/hplsql/src/test/queries/local/create_procedure.sql
@@ -0,0 +1,9 @@
+CREATE PROCEDURE set_message(IN name STRING, OUT result STRING)
+BEGIN
+  SET result = 'Hello, ' || name || '!';
+END;
+ 
+DECLARE str STRING;
+CALL set_message('world', str);
+PRINT str;
+

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/date.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/date.sql b/hplsql/src/test/queries/local/date.sql
new file mode 100644
index 0000000..2ef4743
--- /dev/null
+++ b/hplsql/src/test/queries/local/date.sql
@@ -0,0 +1,5 @@
+DATE '2014-12-20'
+
+DATE('2015-03-12');
+DATE('2015' || '-03-' || '12');
+DATE(TIMESTAMP '2015-03-12 10:58:34.111');
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/dbms_output.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/dbms_output.sql b/hplsql/src/test/queries/local/dbms_output.sql
new file mode 100644
index 0000000..37d7313
--- /dev/null
+++ b/hplsql/src/test/queries/local/dbms_output.sql
@@ -0,0 +1,6 @@
+DECLARE
+  str VARCHAR(200) DEFAULT 'Hello, world!';
+BEGIN
+  DBMS_OUTPUT.PUT_LINE('Hello, world!');
+  DBMS_OUTPUT.PUT_LINE(str);
+END;

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/declare.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/declare.sql b/hplsql/src/test/queries/local/declare.sql
new file mode 100644
index 0000000..fd02da9
--- /dev/null
+++ b/hplsql/src/test/queries/local/declare.sql
@@ -0,0 +1,16 @@
+DECLARE
+  code CHAR(10);
+  status INT := 1;
+  count SMALLINT = 0;
+  limit INT DEFAULT 100;  
+  f UTL_FILE.FILE_TYPE;
+BEGIN
+  status := 2;
+END;
+  
+DECLARE code CHAR(10);
+DECLARE status, status2 INT DEFAULT 1;
+DECLARE count SMALLINT, limit INT DEFAULT 100;  
+
+DECLARE dt DATE DEFAULT '2015-05-13';
+DECLARE ts TIMESTAMP DEFAULT '2015-05-13 11:10:01';
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/declare_condition.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/declare_condition.sql b/hplsql/src/test/queries/local/declare_condition.sql
new file mode 100644
index 0000000..8739499
--- /dev/null
+++ b/hplsql/src/test/queries/local/declare_condition.sql
@@ -0,0 +1,8 @@
+DECLARE cnt_condition CONDITION;
+DECLARE EXIT HANDLER FOR cnt_condition
+  PRINT 'Condition raised';  
+IF 1 <> 2 THEN
+  SIGNAL cnt_condition;
+END IF;
+PRINT 'Must not be printed 1';
+PRINT 'Must not be printed 2';
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/declare_condition2.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/declare_condition2.sql b/hplsql/src/test/queries/local/declare_condition2.sql
new file mode 100644
index 0000000..d6a6461
--- /dev/null
+++ b/hplsql/src/test/queries/local/declare_condition2.sql
@@ -0,0 +1,10 @@
+DECLARE cnt_condition CONDITION;
+DECLARE CONTINUE HANDLER FOR cnt_condition
+  PRINT 'Wrong condition';  
+DECLARE CONTINUE HANDLER FOR cnt_condition2
+  PRINT 'Condition raised';  
+IF 1 <> 2 THEN
+  SIGNAL cnt_condition2;
+END IF;
+PRINT 'Executed 1';
+PRINT 'Executed 2';
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/decode.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/decode.sql b/hplsql/src/test/queries/local/decode.sql
new file mode 100644
index 0000000..a9f7c0c
--- /dev/null
+++ b/hplsql/src/test/queries/local/decode.sql
@@ -0,0 +1,10 @@
+DECLARE var1 INT DEFAULT 3;
+PRINT DECODE (var1, 1, 'A', 2, 'B', 3, 'C');
+PRINT DECODE (var1, 1, 'A', 2, 'B', 'C');
+
+SET var1 := 1;
+PRINT DECODE (var1, 1, 'A', 2, 'B', 3, 'C');
+
+SET var1 := NULL;
+PRINT DECODE (var1, 1, 'A', 2, 'B', NULL, 'C');
+PRINT DECODE (var1, 1, 'A', 2, 'B', 'C');

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/equal.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/equal.sql b/hplsql/src/test/queries/local/equal.sql
new file mode 100644
index 0000000..0de2801
--- /dev/null
+++ b/hplsql/src/test/queries/local/equal.sql
@@ -0,0 +1,55 @@
+PRINT 'Case 1 = 1';
+IF 1 = 1 THEN
+  PRINT 'Equal - Correct';
+ELSE
+  PRINT 'Not equal - Incorrect';
+END IF;
+
+PRINT 'Case 1 == 1';
+IF 1 == 1 THEN
+  PRINT 'Equal - Correct';
+ELSE
+  PRINT 'Not equal - Incorrect';
+END IF;
+
+PRINT 'Case 1 <> 3';
+IF 1 <> 3 THEN
+  PRINT 'Not equal - Correct';
+ELSE
+  PRINT 'Equal - Incorrect';
+END IF;
+
+PRINT 'Case 1 != 3';
+IF 1 != 3 THEN
+  PRINT 'Not equal - Correct';
+ELSE
+  PRINT 'Equal - Incorrect';
+END IF;
+
+PRINT 'Case 3 > 1';
+IF 3 > 1 THEN
+  PRINT 'Greater - Correct';
+ELSE
+  PRINT 'Greater - Incorrect';
+END IF;
+
+PRINT 'Case 1 < 3';
+IF 1 < 3 THEN
+  PRINT 'Less - Correct';
+ELSE
+  PRINT 'Less - Incorrect';
+END IF;
+
+PRINT 'Case 3 >= 1';
+IF 3 >= 1 THEN
+  PRINT 'Greater or equal - Correct';
+ELSE
+  PRINT 'Greater or equal - Incorrect';
+END IF;
+
+PRINT 'Case 1 <= 3';
+IF 1 <= 3 THEN
+  PRINT 'Less or equal - Correct';
+ELSE
+  PRINT 'Less or equal - Incorrect';
+END IF;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/exception.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/exception.sql b/hplsql/src/test/queries/local/exception.sql
new file mode 100644
index 0000000..7ce7377
--- /dev/null
+++ b/hplsql/src/test/queries/local/exception.sql
@@ -0,0 +1,14 @@
+BEGIN
+  PRINT 'Correct';
+  WHILE 1=1 THEN
+    FETCH cur INTO v;
+    PRINT 'Incorrect - unreachable code, unknown cursor name, exception must be raised';
+  END WHILE;
+EXCEPTION WHEN OTHERS THEN
+  PRINT 'Correct';
+  PRINT 'Correct';
+  PRINT 'Correct - Exception raised';   
+  WHEN NO_DATA_FOUND THEN
+  PRINT 'Correct';
+END 
+

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/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
new file mode 100644
index 0000000..3394da8
--- /dev/null
+++ b/hplsql/src/test/queries/local/exception2.sql
@@ -0,0 +1,10 @@
+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/7338d8e1/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
new file mode 100644
index 0000000..a12b853
--- /dev/null
+++ b/hplsql/src/test/queries/local/exception3.sql
@@ -0,0 +1,5 @@
+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/7338d8e1/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
new file mode 100644
index 0000000..38d89b5
--- /dev/null
+++ b/hplsql/src/test/queries/local/exception4.sql
@@ -0,0 +1,7 @@
+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/7338d8e1/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
new file mode 100644
index 0000000..6232984
--- /dev/null
+++ b/hplsql/src/test/queries/local/exception5.sql
@@ -0,0 +1,10 @@
+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/7338d8e1/hplsql/src/test/queries/local/exit.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/exit.sql b/hplsql/src/test/queries/local/exit.sql
new file mode 100644
index 0000000..d0e432b
--- /dev/null
+++ b/hplsql/src/test/queries/local/exit.sql
@@ -0,0 +1,31 @@
+DECLARE count INT DEFAULT 3;
+
+WHILE 1=1 LOOP
+  PRINT 'Start of while block';
+  PRINT count;
+  count := count - 1;
+  EXIT WHEN count = 0;
+  PRINT 'End of while block';
+END LOOP;
+
+count := 3;
+
+<<lbl>>
+WHILE 1=1 LOOP
+  PRINT 'Start of outer while block';
+  
+  WHILE 1=1 LOOP
+    PRINT 'Start of 1st inner while block';
+    EXIT;
+    PRINT 'End of 1st inner while block (NEVER SHOWN)';
+  END LOOP;
+  
+  <<lbl2>>
+  WHILE 1=1 LOOP
+    PRINT 'Start of 2nd inner while block';
+    EXIT lbl;
+    PRINT 'End of 2nd inner while block (NEVER SHOWN)';
+  END LOOP;
+  PRINT 'End of outer while block (NEVER SHOWN)';
+END LOOP;
+PRINT 'End of script';
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/expr.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/expr.sql b/hplsql/src/test/queries/local/expr.sql
new file mode 100644
index 0000000..33388a2
--- /dev/null
+++ b/hplsql/src/test/queries/local/expr.sql
@@ -0,0 +1,21 @@
+PRINT 'a' || 'b';
+PRINT 'a' || 1 || 'b';
+PRINT 1 || 'a' || 'b';
+PRINT 'a' || null || 'b';
+PRINT null || 'a' || 'b';
+PRINT null || null;
+
+DECLARE c INT;
+
+PRINT 'Integer increment'; 
+c := 3;
+c := c + 1;
+PRINT c;
+
+PRINT 'Integer decrement'; 
+c := 3;
+c := c - 1;
+PRINT c; 
+
+PRINT NVL(null - 3, 'Correct');
+PRINT NVL(null + 3, 'Correct');
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/for_range.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/for_range.sql b/hplsql/src/test/queries/local/for_range.sql
new file mode 100644
index 0000000..b7af115
--- /dev/null
+++ b/hplsql/src/test/queries/local/for_range.sql
@@ -0,0 +1,20 @@
+DECLARE i INT = 3;
+PRINT i;
+
+FOR i IN 1..10 LOOP
+  PRINT i;
+END LOOP;
+
+PRINT i;
+
+FOR i IN REVERSE 10..1 LOOP
+  PRINT i;
+END LOOP;
+
+PRINT i;
+
+FOR i IN 1..10 BY 2 LOOP
+  PRINT i;
+END LOOP;
+
+PRINT i;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/if.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/if.sql b/hplsql/src/test/queries/local/if.sql
new file mode 100644
index 0000000..2de3045
--- /dev/null
+++ b/hplsql/src/test/queries/local/if.sql
@@ -0,0 +1,68 @@
+DECLARE state VARCHAR;
+DECLARE count INT;
+
+SET state = 'CA';
+SET count = 1;
+
+/*IF count = 1 THEN
+  PRINT 'True block - Correct';
+END IF;*/
+
+IF state = 'CA' THEN
+  PRINT 'True block - Correct';
+ELSE
+  PRINT 'False block - Incorrect';
+END IF;
+
+IF state = 'MA' THEN
+  PRINT 'True block - Incorrect';
+ELSE
+  PRINT 'False block - Correct';
+END IF;
+
+IF count = 4 THEN
+  PRINT 'True block - Incorrect';  
+ELSIF count = 3 THEN
+  PRINT 'True block - Incorrect';  
+ELSIF count = 2 THEN
+  PRINT 'True block - Incorrect';  
+ELSE
+  PRINT 'False block - Correct'; 
+END IF;
+
+IF count = 3 THEN
+  PRINT 'True block - Incorrect';  
+ELSIF count = 2 THEN
+  PRINT 'True block - Incorrect';  
+ELSIF count = 1 THEN
+  PRINT 'True block - Correct';  
+ELSE
+  PRINT 'False block - Incorrect'; 
+END IF;
+
+PRINT 'IS NOT NULL AND BETWEEN';
+IF 1 IS NOT NULL AND 1 BETWEEN 0 AND 100 THEN
+  PRINT 'True block - Correct';  
+ELSE
+  PRINT 'False block - Incorrect'; 
+END IF;
+
+PRINT 'Transact-SQL - Single statement';
+
+IF state = 'CA'
+  PRINT 'True block - Correct';  
+ELSE 
+  PRINT 'False block - Incorrect'; 
+
+PRINT 'Transact-SQL - BEGIN-END block';
+  
+IF state = 'CA'
+BEGIN
+  PRINT 'True block - Correct'; 
+  PRINT 'True block - Correct'; 
+END
+ELSE 
+BEGIN
+  PRINT 'False block - Incorrect'; 
+  PRINT 'False block - Incorrect'; 
+END  
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/instr.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/instr.sql b/hplsql/src/test/queries/local/instr.sql
new file mode 100644
index 0000000..9cd8dca
--- /dev/null
+++ b/hplsql/src/test/queries/local/instr.sql
@@ -0,0 +1,49 @@
+IF INSTR('abc', 'b') = 2 THEN
+  PRINT 'Correct';
+ELSE
+  PRINT 'Failed';
+END IF;
+
+IF INSTR('abcabc', 'b', 3) = 5 THEN
+  PRINT 'Correct';
+ELSE
+  PRINT 'Failed';
+END IF;
+
+IF INSTR('abcabcabc', 'b', 3, 2) = 8 THEN
+  PRINT 'Correct';
+ELSE
+  PRINT 'Failed';
+END IF; 
+
+IF INSTR('abcabcabc', 'b', -3) = 5 THEN
+  PRINT 'Correct';
+ELSE
+  PRINT 'Failed';
+END IF; 
+
+IF INSTR('abcabcabc', 'b', -3, 2) = 2 THEN
+  PRINT 'Correct';
+ELSE
+  PRINT 'Failed';
+END IF;
+
+DECLARE c STRING;
+
+IF INSTR(c, 'b') IS NULL THEN
+  PRINT 'Correct';
+ELSE
+  PRINT 'Failed';
+END IF;
+
+IF INSTR(NULL, 'b') IS NULL THEN
+  PRINT 'Correct';
+ELSE
+  PRINT 'Failed';
+END IF;
+
+IF INSTR('', 'b') = 0 THEN
+  PRINT 'Correct';
+ELSE
+  PRINT 'Failed';
+END IF;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/interval.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/interval.sql b/hplsql/src/test/queries/local/interval.sql
new file mode 100644
index 0000000..7962f2d
--- /dev/null
+++ b/hplsql/src/test/queries/local/interval.sql
@@ -0,0 +1,15 @@
+DATE '2015-03-12' + 1 DAY;
+TIMESTAMP '2015-03-12' + 1 DAY;
+TIMESTAMP '2015-03-12 10:10:10.000' + 1 MICROSECOND;
+
+DATE '2015-03-12' + NVL(NULL, 3) DAYS;
+TIMESTAMP '2015-03-12' + NVL(NULL, 3) DAYS;
+
+DATE '2015-03-12' - 1 DAY;
+TIMESTAMP '2015-03-12' - 1 DAY;
+TIMESTAMP '2015-03-12 10:10:10.000' - 1 MICROSECOND;
+
+DATE '2015-03-12' - NVL(NULL, 3) DAYS;
+TIMESTAMP '2015-03-12' - NVL(NULL, 3) DAYS;
+
+TIMESTAMP '2015-03-12' - 1 DAY - 1 MICROSECOND;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/lang.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/lang.sql b/hplsql/src/test/queries/local/lang.sql
new file mode 100644
index 0000000..56f8c33
--- /dev/null
+++ b/hplsql/src/test/queries/local/lang.sql
@@ -0,0 +1,57 @@
+-- Integer literals
++1;
+1;
+0;
+-1;
+
+-- Decimal literals
+1.0;
++1.0;
+-1.0;
+-- 1.;
+-- +1.;
+-- -1.;
+-- .1;
+-- +.1;
+-- -.1;
+
+-- Identifiers
+declare abc int;
+declare abc.abc int;
+declare abc . abc1 int;
+declare "abc" int;
+declare "abc".abc int;
+declare "abc"."abc" int;
+declare "abc" . "abc1" int;
+declare [abc] int;
+declare [abc].abc int;
+declare [abc].[abc] int;
+declare [abc] . [abc1] int;
+declare `abc` int;
+declare `abc`.abc int;
+declare `abc`.`abc` int;
+declare `abc` . `abc1` int;
+declare :new.abc int;
+declare @abc int;
+declare _abc int;
+declare #abc int;
+declare ##abc int;
+declare $abc int;
+declare abc_9 int;
+
+-- Operators and expressions
++1 + 1;                 -- 2
+1 + 1;                  -- 2
+1 + -1;                 -- 0
+-- 'a' + 'b';              -- ab    
+-- 'a''b' + 'c';           -- ab''c   
+-- 'a\'b' + 'c';           -- ab\'c   
+-- 1 + '1'                 -- 2        
+-- '1' + 1                 -- 2
+-- 1 + 'a'                 -- 1a     
+-- 'a' + 1                 -- a1
+
+-1 - 1;   -- -2
+-1 - -1;  -- 0
+
+ 
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/leave.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/leave.sql b/hplsql/src/test/queries/local/leave.sql
new file mode 100644
index 0000000..a4fc0d5
--- /dev/null
+++ b/hplsql/src/test/queries/local/leave.sql
@@ -0,0 +1,33 @@
+DECLARE count INT DEFAULT 3;
+lbl:
+WHILE 1=1 DO
+  PRINT 'Start of while block';
+  PRINT count;
+  SET count = count - 1;
+  IF count = 0 THEN
+    LEAVE lbl;
+  END IF;
+  PRINT 'End of while block';
+END WHILE;
+
+SET count = 3;
+
+lbl3:
+WHILE 1=1 DO
+  PRINT 'Start of outer while block';
+  
+  lbl1:
+  WHILE 1=1 DO
+    PRINT 'Start of 1st inner while block';
+    LEAVE lbl1;
+    PRINT 'End of 1st inner while block (NEVER SHOWN)';
+  END WHILE;
+  
+  lbl2:
+  WHILE 1=1 DO
+    PRINT 'Start of 2nd inner while block';
+    LEAVE lbl3;
+    PRINT 'End of 2nd inner while block (NEVER SHOWN)';
+  END WHILE;
+  PRINT 'End of outer while block (NEVER SHOWN)';
+END WHILE;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/len.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/len.sql b/hplsql/src/test/queries/local/len.sql
new file mode 100644
index 0000000..9851c49
--- /dev/null
+++ b/hplsql/src/test/queries/local/len.sql
@@ -0,0 +1 @@
+LEN('Abc ');
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/length.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/length.sql b/hplsql/src/test/queries/local/length.sql
new file mode 100644
index 0000000..42cf3cc
--- /dev/null
+++ b/hplsql/src/test/queries/local/length.sql
@@ -0,0 +1 @@
+LENGTH('Abc ');
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/lower.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/lower.sql b/hplsql/src/test/queries/local/lower.sql
new file mode 100644
index 0000000..f29b0e9
--- /dev/null
+++ b/hplsql/src/test/queries/local/lower.sql
@@ -0,0 +1 @@
+LOWER('ABC');
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/nvl.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/nvl.sql b/hplsql/src/test/queries/local/nvl.sql
new file mode 100644
index 0000000..1a843bc
--- /dev/null
+++ b/hplsql/src/test/queries/local/nvl.sql
@@ -0,0 +1,4 @@
+NVL('First non-null', 1);
+NVL(NULL, 'First non-null');
+NVL(NULL, 'First non-null', 1);
+NVL(NULL, NULL, 'First non-null', 1);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/nvl2.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/nvl2.sql b/hplsql/src/test/queries/local/nvl2.sql
new file mode 100644
index 0000000..70eeccb
--- /dev/null
+++ b/hplsql/src/test/queries/local/nvl2.sql
@@ -0,0 +1,2 @@
+NVL2('A', 'Correct', 'FAILED');
+NVL2(NULL, 'FAILED', 'Correct');

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/print.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/print.sql b/hplsql/src/test/queries/local/print.sql
new file mode 100644
index 0000000..095682b
--- /dev/null
+++ b/hplsql/src/test/queries/local/print.sql
@@ -0,0 +1,5 @@
+PRINT 1;
+PRINT 'abc';
+PRINT ('abc');
+
+

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/return.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/return.sql b/hplsql/src/test/queries/local/return.sql
new file mode 100644
index 0000000..c52e5c5
--- /dev/null
+++ b/hplsql/src/test/queries/local/return.sql
@@ -0,0 +1,3 @@
+PRINT 'Before return';
+RETURN; 
+PRINT 'Unreachable code';
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/seterror.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/seterror.sql b/hplsql/src/test/queries/local/seterror.sql
new file mode 100644
index 0000000..4705677
--- /dev/null
+++ b/hplsql/src/test/queries/local/seterror.sql
@@ -0,0 +1,10 @@
+BEGIN
+SET plhql.onerror = SETERROR;
+ 
+HOST 'abcd';
+IF HOSTCODE <> 0 THEN
+  PRINT 'Correct';
+END IF;
+EXCEPTION WHEN OTHERS THEN
+  PRINT 'FAILED';
+END
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/sub.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/sub.sql b/hplsql/src/test/queries/local/sub.sql
new file mode 100644
index 0000000..a32bef7
--- /dev/null
+++ b/hplsql/src/test/queries/local/sub.sql
@@ -0,0 +1 @@
+DATE '2015-01-01' - 1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/substr.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/substr.sql b/hplsql/src/test/queries/local/substr.sql
new file mode 100644
index 0000000..7785e39
--- /dev/null
+++ b/hplsql/src/test/queries/local/substr.sql
@@ -0,0 +1,2 @@
+SUBSTR('FAILED Correct', 8);
+SUBSTR('FAILED Correct FAILED', 8, 7); 
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/substring.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/substring.sql b/hplsql/src/test/queries/local/substring.sql
new file mode 100644
index 0000000..c94a191
--- /dev/null
+++ b/hplsql/src/test/queries/local/substring.sql
@@ -0,0 +1,8 @@
+SUBSTRING('FAILED Correct', 8);
+SUBSTRING('FAILED Correct FAILED', 8, 7); 
+
+SUBSTRING('FAILED Correct' FROM 8);
+SUBSTRING('FAILED Correct FAILED' FROM 8 FOR 7); 
+
+SUBSTRING('', 8);
+SUBSTRING(NULL, 8);

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/timestamp.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/timestamp.sql b/hplsql/src/test/queries/local/timestamp.sql
new file mode 100644
index 0000000..2971cea
--- /dev/null
+++ b/hplsql/src/test/queries/local/timestamp.sql
@@ -0,0 +1,4 @@
+TIMESTAMP '2015-03-03 11:39:31.123456';
+TIMESTAMP '2015-03-03 11:39:31.123';
+TIMESTAMP '2015-03-03 11:39:31';
+TIMESTAMP '2015-03-03-11.39.31.123';
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/timestamp_iso.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/timestamp_iso.sql b/hplsql/src/test/queries/local/timestamp_iso.sql
new file mode 100644
index 0000000..9bcdfe0
--- /dev/null
+++ b/hplsql/src/test/queries/local/timestamp_iso.sql
@@ -0,0 +1,2 @@
+TIMESTAMP_ISO('2015-03-12');
+TIMESTAMP_ISO(DATE '2015-03-12');
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/to_char.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/to_char.sql b/hplsql/src/test/queries/local/to_char.sql
new file mode 100644
index 0000000..339c7d6
--- /dev/null
+++ b/hplsql/src/test/queries/local/to_char.sql
@@ -0,0 +1 @@
+TO_CHAR(DATE '2015-04-02')
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/to_timestamp.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/to_timestamp.sql b/hplsql/src/test/queries/local/to_timestamp.sql
new file mode 100644
index 0000000..c18f1f4
--- /dev/null
+++ b/hplsql/src/test/queries/local/to_timestamp.sql
@@ -0,0 +1,5 @@
+TO_TIMESTAMP('2015-04-02', 'YYYY-MM-DD');
+TO_TIMESTAMP('2015-04-02', 'yyyy-mm-dd');
+TO_TIMESTAMP('04/02/2015', 'mm/dd/yyyy');
+
+TO_TIMESTAMP('2015-04-02 13:51:31', 'YYYY-MM-DD HH24:MI:SS');
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/trim.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/trim.sql b/hplsql/src/test/queries/local/trim.sql
new file mode 100644
index 0000000..f8a2978
--- /dev/null
+++ b/hplsql/src/test/queries/local/trim.sql
@@ -0,0 +1 @@
+'#' || TRIM(' Hello ') || '#';
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/twopipes.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/twopipes.sql b/hplsql/src/test/queries/local/twopipes.sql
new file mode 100644
index 0000000..c1d6f1d
--- /dev/null
+++ b/hplsql/src/test/queries/local/twopipes.sql
@@ -0,0 +1 @@
+'a' || 'b' || 'c'
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/upper.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/upper.sql b/hplsql/src/test/queries/local/upper.sql
new file mode 100644
index 0000000..9b3b522
--- /dev/null
+++ b/hplsql/src/test/queries/local/upper.sql
@@ -0,0 +1 @@
+UPPER('abc');
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/values_into.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/values_into.sql b/hplsql/src/test/queries/local/values_into.sql
new file mode 100644
index 0000000..e49894a
--- /dev/null
+++ b/hplsql/src/test/queries/local/values_into.sql
@@ -0,0 +1,6 @@
+VALUES 'A' INTO code;
+VALUES (0, 100) INTO (limit, count); 
+
+PRINT code;
+PRINT count;
+PRINT limit;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/queries/local/while.sql
----------------------------------------------------------------------
diff --git a/hplsql/src/test/queries/local/while.sql b/hplsql/src/test/queries/local/while.sql
new file mode 100644
index 0000000..2dc4b54
--- /dev/null
+++ b/hplsql/src/test/queries/local/while.sql
@@ -0,0 +1,20 @@
+DECLARE count INT DEFAULT 7;
+
+WHILE count <> 0 LOOP
+  PRINT count;
+  count := count - 1;
+END LOOP;
+
+SET count = 7;
+
+WHILE count <> 0 DO
+  PRINT count;
+  SET count = count - 1;
+END WHILE;
+
+SET count = 7;
+
+WHILE count <> 0 BEGIN
+  PRINT count;
+  SET count = count - 1;
+END
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/add.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/add.out.txt b/hplsql/src/test/results/local/add.out.txt
new file mode 100644
index 0000000..37a195b
--- /dev/null
+++ b/hplsql/src/test/results/local/add.out.txt
@@ -0,0 +1,2 @@
+2015-01-01
+2015-01-01

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/assign.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/assign.out.txt b/hplsql/src/test/results/local/assign.out.txt
new file mode 100644
index 0000000..c01e270
--- /dev/null
+++ b/hplsql/src/test/results/local/assign.out.txt
@@ -0,0 +1,8 @@
+Ln:1 SET code = 'A'
+Ln:2 SET status = 1
+Ln:3 SET count = 0
+Ln:5 SET code = 'A'
+Ln:6 SET status = 1
+Ln:6 SET count = 0
+Ln:7 SET count = 0
+Ln:7 SET limit = 100

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/bool_expr.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/bool_expr.out.txt b/hplsql/src/test/results/local/bool_expr.out.txt
new file mode 100644
index 0000000..514f324
--- /dev/null
+++ b/hplsql/src/test/results/local/bool_expr.out.txt
@@ -0,0 +1,32 @@
+Ln:1 IF
+Ln:1 IF TRUE executed
+Ln:2 PRINT
+Correct
+Ln:7 IF
+Ln:7 IF TRUE executed
+Ln:8 PRINT
+Correct
+Ln:13 IF
+Ln:13 IF TRUE executed
+Ln:14 PRINT
+Correct
+Ln:19 IF
+Ln:19 ELSE executed
+Ln:22 PRINT
+Correct
+Ln:25 IF
+Ln:25 IF TRUE executed
+Ln:26 PRINT
+Correct
+Ln:31 IF
+Ln:31 IF TRUE executed
+Ln:32 PRINT
+Correct
+Ln:37 IF
+Ln:37 IF TRUE executed
+Ln:38 PRINT
+Correct
+Ln:43 IF
+Ln:43 ELSE executed
+Ln:46 PRINT
+Correct
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/break.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/break.out.txt b/hplsql/src/test/results/local/break.out.txt
new file mode 100644
index 0000000..cf907df
--- /dev/null
+++ b/hplsql/src/test/results/local/break.out.txt
@@ -0,0 +1,29 @@
+Ln:1 DECLARE count INT = 3
+Ln:2 WHILE - ENTERED
+Ln:3 PRINT
+Start of while block
+Ln:4 PRINT
+3
+Ln:5 SET count = 2
+Ln:6 IF
+Ln:8 PRINT
+End of while block
+Ln:3 PRINT
+Start of while block
+Ln:4 PRINT
+2
+Ln:5 SET count = 1
+Ln:6 IF
+Ln:8 PRINT
+End of while block
+Ln:3 PRINT
+Start of while block
+Ln:4 PRINT
+1
+Ln:5 SET count = 0
+Ln:6 IF
+Ln:6 IF TRUE executed
+Ln:7 BREAK
+Ln:2 WHILE - LEFT
+Ln:10 PRINT
+End of script
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/case.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/case.out.txt b/hplsql/src/test/results/local/case.out.txt
new file mode 100644
index 0000000..6062a1f
--- /dev/null
+++ b/hplsql/src/test/results/local/case.out.txt
@@ -0,0 +1,12 @@
+Ln:1 PRINT
+Correct
+Ln:8 PRINT
+Correct
+Ln:14 PRINT
+Correct
+Ln:19 PRINT
+Correct
+Ln:26 PRINT
+Correct
+Ln:32 PRINT
+Correct

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/cast.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/cast.out.txt b/hplsql/src/test/results/local/cast.out.txt
new file mode 100644
index 0000000..f3de493
--- /dev/null
+++ b/hplsql/src/test/results/local/cast.out.txt
@@ -0,0 +1,8 @@
+Ln:1 FUNC CAST
+A
+Ln:2 FUNC CAST
+Ab
+Ln:3 FUNC CAST
+Abc
+Ln:4 FUNC CAST
+2015-03-12

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/char.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/char.out.txt b/hplsql/src/test/results/local/char.out.txt
new file mode 100644
index 0000000..83b33d2
--- /dev/null
+++ b/hplsql/src/test/results/local/char.out.txt
@@ -0,0 +1 @@
+1000

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/coalesce.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/coalesce.out.txt b/hplsql/src/test/results/local/coalesce.out.txt
new file mode 100644
index 0000000..a111c85
--- /dev/null
+++ b/hplsql/src/test/results/local/coalesce.out.txt
@@ -0,0 +1,4 @@
+First non-null
+First non-null
+First non-null
+First non-null

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/concat.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/concat.out.txt b/hplsql/src/test/results/local/concat.out.txt
new file mode 100644
index 0000000..cdddd69
--- /dev/null
+++ b/hplsql/src/test/results/local/concat.out.txt
@@ -0,0 +1,2 @@
+abc
+NULL Value

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/create_function.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/create_function.out.txt b/hplsql/src/test/results/local/create_function.out.txt
new file mode 100644
index 0000000..b996ab4
--- /dev/null
+++ b/hplsql/src/test/results/local/create_function.out.txt
@@ -0,0 +1,9 @@
+Ln:1 CREATE FUNCTION hello
+Ln:10 PRINT
+Ln:10 EXEC FUNCTION hello
+Ln:4 PRINT
+Start
+Ln:5 RETURN
+Hello, world!
+Ln:11 PRINT
+End of script

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/create_function2.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/create_function2.out.txt b/hplsql/src/test/results/local/create_function2.out.txt
new file mode 100644
index 0000000..c8fc993
--- /dev/null
+++ b/hplsql/src/test/results/local/create_function2.out.txt
@@ -0,0 +1,10 @@
+Ln:1 CREATE FUNCTION hello2
+Ln:10 PRINT
+Ln:10 EXEC FUNCTION hello2
+Ln:10 SET PARAM text = world
+Ln:4 PRINT
+Start
+Ln:5 RETURN
+Hello, world!
+Ln:11 PRINT
+End of script

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/create_procedure.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/create_procedure.out.txt b/hplsql/src/test/results/local/create_procedure.out.txt
new file mode 100644
index 0000000..1f86916
--- /dev/null
+++ b/hplsql/src/test/results/local/create_procedure.out.txt
@@ -0,0 +1,8 @@
+Ln:1 CREATE PROCEDURE set_message
+Ln:6 DECLARE str STRING
+Ln:7 EXEC PROCEDURE set_message
+Ln:7 SET PARAM name = world
+Ln:7 SET PARAM result = null
+Ln:3 SET result = 'Hello, world!'
+Ln:8 PRINT
+Hello, world!
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/date.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/date.out.txt b/hplsql/src/test/results/local/date.out.txt
new file mode 100644
index 0000000..118bd29
--- /dev/null
+++ b/hplsql/src/test/results/local/date.out.txt
@@ -0,0 +1,4 @@
+2014-12-20
+2015-03-12
+2015-03-12
+2015-03-12

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/dbms_output.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/dbms_output.out.txt b/hplsql/src/test/results/local/dbms_output.out.txt
new file mode 100644
index 0000000..b6ed0e0
--- /dev/null
+++ b/hplsql/src/test/results/local/dbms_output.out.txt
@@ -0,0 +1,3 @@
+Ln:2 DECLARE str VARCHAR = 'Hello, world!'
+Hello, world!
+Hello, world!

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/declare.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/declare.out.txt b/hplsql/src/test/results/local/declare.out.txt
new file mode 100644
index 0000000..2b7794e
--- /dev/null
+++ b/hplsql/src/test/results/local/declare.out.txt
@@ -0,0 +1,13 @@
+Ln:2 DECLARE code CHAR
+Ln:3 DECLARE status INT = 1
+Ln:4 DECLARE count SMALLINT = NULL
+Ln:5 DECLARE limit INT = 100
+Ln:6 DECLARE f UTL_FILE.FILE_TYPE
+Ln:8 SET status = 2
+Ln:11 DECLARE code CHAR
+Ln:12 DECLARE status INT = 1
+Ln:12 DECLARE status2 INT = 1
+Ln:13 DECLARE count SMALLINT
+Ln:13 DECLARE limit INT = 100
+Ln:15 DECLARE dt DATE = 2015-05-13
+Ln:16 DECLARE ts TIMESTAMP = 2015-05-13 11:10:01

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/declare_condition.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/declare_condition.out.txt b/hplsql/src/test/results/local/declare_condition.out.txt
new file mode 100644
index 0000000..4633c8d
--- /dev/null
+++ b/hplsql/src/test/results/local/declare_condition.out.txt
@@ -0,0 +1,7 @@
+Ln:2 DECLARE HANDLER
+Ln:4 IF
+Ln:4 IF TRUE executed
+Ln:5 SIGNAL
+Ln:2 EXIT HANDLER
+Ln:3 PRINT
+Condition raised
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/declare_condition2.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/declare_condition2.out.txt b/hplsql/src/test/results/local/declare_condition2.out.txt
new file mode 100644
index 0000000..67da39d
--- /dev/null
+++ b/hplsql/src/test/results/local/declare_condition2.out.txt
@@ -0,0 +1,12 @@
+Ln:2 DECLARE HANDLER
+Ln:4 DECLARE HANDLER
+Ln:6 IF
+Ln:6 IF TRUE executed
+Ln:7 SIGNAL
+Ln:4 CONTINUE HANDLER
+Ln:5 PRINT
+Condition raised
+Ln:9 PRINT
+Executed 1
+Ln:10 PRINT
+Executed 2
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/decode.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/decode.out.txt b/hplsql/src/test/results/local/decode.out.txt
new file mode 100644
index 0000000..39b01bc
--- /dev/null
+++ b/hplsql/src/test/results/local/decode.out.txt
@@ -0,0 +1,13 @@
+Ln:1 DECLARE var1 INT = 3
+Ln:2 PRINT
+C
+Ln:3 PRINT
+C
+Ln:5 SET var1 = 1
+Ln:6 PRINT
+A
+Ln:8 SET var1 = NULL
+Ln:9 PRINT
+C
+Ln:10 PRINT
+C

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/equal.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/equal.out.txt b/hplsql/src/test/results/local/equal.out.txt
new file mode 100644
index 0000000..9c02e38
--- /dev/null
+++ b/hplsql/src/test/results/local/equal.out.txt
@@ -0,0 +1,48 @@
+Ln:1 PRINT
+Case 1 = 1
+Ln:2 IF
+Ln:2 IF TRUE executed
+Ln:3 PRINT
+Equal - Correct
+Ln:8 PRINT
+Case 1 == 1
+Ln:9 IF
+Ln:9 IF TRUE executed
+Ln:10 PRINT
+Equal - Correct
+Ln:15 PRINT
+Case 1 <> 3
+Ln:16 IF
+Ln:16 IF TRUE executed
+Ln:17 PRINT
+Not equal - Correct
+Ln:22 PRINT
+Case 1 != 3
+Ln:23 IF
+Ln:23 IF TRUE executed
+Ln:24 PRINT
+Not equal - Correct
+Ln:29 PRINT
+Case 3 > 1
+Ln:30 IF
+Ln:30 IF TRUE executed
+Ln:31 PRINT
+Greater - Correct
+Ln:36 PRINT
+Case 1 < 3
+Ln:37 IF
+Ln:37 IF TRUE executed
+Ln:38 PRINT
+Less - Correct
+Ln:43 PRINT
+Case 3 >= 1
+Ln:44 IF
+Ln:44 IF TRUE executed
+Ln:45 PRINT
+Greater or equal - Correct
+Ln:50 PRINT
+Case 1 <= 3
+Ln:51 IF
+Ln:51 IF TRUE executed
+Ln:52 PRINT
+Less or equal - Correct
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/exception.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/exception.out.txt b/hplsql/src/test/results/local/exception.out.txt
new file mode 100644
index 0000000..5de7998
--- /dev/null
+++ b/hplsql/src/test/results/local/exception.out.txt
@@ -0,0 +1,13 @@
+Ln:2 PRINT
+Correct
+Ln:3 WHILE - ENTERED
+Ln:4 FETCH
+Ln:4 Cursor not found: cur
+Ln:3 WHILE - LEFT
+Ln:7 EXCEPTION HANDLER
+Ln:8 PRINT
+Correct
+Ln:9 PRINT
+Correct
+Ln:10 PRINT
+Correct - Exception raised
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/exception2.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/exception2.out.txt b/hplsql/src/test/results/local/exception2.out.txt
new file mode 100644
index 0000000..f56a326
--- /dev/null
+++ b/hplsql/src/test/results/local/exception2.out.txt
@@ -0,0 +1,5 @@
+Ln:2 DECLARE v VARCHAR
+Ln:4 OPEN
+Ln:4 cur: SELECT c1 FROM t1
+Ln:7 EXCEPTION HANDLER
+Error

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/exit.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/exit.out.txt b/hplsql/src/test/results/local/exit.out.txt
new file mode 100644
index 0000000..0352275
--- /dev/null
+++ b/hplsql/src/test/results/local/exit.out.txt
@@ -0,0 +1,42 @@
+Ln:1 DECLARE count INT = 3
+Ln:3 WHILE - ENTERED
+Ln:4 PRINT
+Start of while block
+Ln:5 PRINT
+3
+Ln:6 SET count = 2
+Ln:7 EXIT
+Ln:8 PRINT
+End of while block
+Ln:4 PRINT
+Start of while block
+Ln:5 PRINT
+2
+Ln:6 SET count = 1
+Ln:7 EXIT
+Ln:8 PRINT
+End of while block
+Ln:4 PRINT
+Start of while block
+Ln:5 PRINT
+1
+Ln:6 SET count = 0
+Ln:7 EXIT
+Ln:3 WHILE - LEFT
+Ln:11 SET count = 3
+Ln:14 WHILE - ENTERED
+Ln:15 PRINT
+Start of outer while block
+Ln:17 WHILE - ENTERED
+Ln:18 PRINT
+Start of 1st inner while block
+Ln:19 EXIT
+Ln:17 WHILE - LEFT
+Ln:24 WHILE - ENTERED
+Ln:25 PRINT
+Start of 2nd inner while block
+Ln:26 EXIT
+Ln:24 WHILE - LEFT
+Ln:14 WHILE - LEFT
+Ln:31 PRINT
+End of script

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/expr.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/expr.out.txt b/hplsql/src/test/results/local/expr.out.txt
new file mode 100644
index 0000000..a5377e3
--- /dev/null
+++ b/hplsql/src/test/results/local/expr.out.txt
@@ -0,0 +1,29 @@
+Ln:1 PRINT
+ab
+Ln:2 PRINT
+a1b
+Ln:3 PRINT
+1ab
+Ln:4 PRINT
+ab
+Ln:5 PRINT
+ab
+Ln:6 PRINT
+null
+Ln:8 DECLARE c INT
+Ln:10 PRINT
+Integer increment
+Ln:11 SET c = 3
+Ln:12 SET c = 4
+Ln:13 PRINT
+4
+Ln:15 PRINT
+Integer decrement
+Ln:16 SET c = 3
+Ln:17 SET c = 2
+Ln:18 PRINT
+2
+Ln:20 PRINT
+Correct
+Ln:21 PRINT
+Correct
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/for_range.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/for_range.out.txt b/hplsql/src/test/results/local/for_range.out.txt
new file mode 100644
index 0000000..dc29c95
--- /dev/null
+++ b/hplsql/src/test/results/local/for_range.out.txt
@@ -0,0 +1,65 @@
+Ln:1 DECLARE i INT = 3
+Ln:2 PRINT
+3
+Ln:4 FOR RANGE - ENTERED
+Ln:5 PRINT
+1
+Ln:5 PRINT
+2
+Ln:5 PRINT
+3
+Ln:5 PRINT
+4
+Ln:5 PRINT
+5
+Ln:5 PRINT
+6
+Ln:5 PRINT
+7
+Ln:5 PRINT
+8
+Ln:5 PRINT
+9
+Ln:5 PRINT
+10
+Ln:4 FOR RANGE - LEFT
+Ln:8 PRINT
+3
+Ln:10 FOR RANGE - ENTERED
+Ln:11 PRINT
+10
+Ln:11 PRINT
+9
+Ln:11 PRINT
+8
+Ln:11 PRINT
+7
+Ln:11 PRINT
+6
+Ln:11 PRINT
+5
+Ln:11 PRINT
+4
+Ln:11 PRINT
+3
+Ln:11 PRINT
+2
+Ln:11 PRINT
+1
+Ln:10 FOR RANGE - LEFT
+Ln:14 PRINT
+3
+Ln:16 FOR RANGE - ENTERED
+Ln:17 PRINT
+1
+Ln:17 PRINT
+3
+Ln:17 PRINT
+5
+Ln:17 PRINT
+7
+Ln:17 PRINT
+9
+Ln:16 FOR RANGE - LEFT
+Ln:20 PRINT
+3

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/if.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/if.out.txt b/hplsql/src/test/results/local/if.out.txt
new file mode 100644
index 0000000..1da8142
--- /dev/null
+++ b/hplsql/src/test/results/local/if.out.txt
@@ -0,0 +1,40 @@
+Ln:1 DECLARE state VARCHAR
+Ln:2 DECLARE count INT
+Ln:4 SET state = 'CA'
+Ln:5 SET count = 1
+Ln:11 IF
+Ln:11 IF TRUE executed
+Ln:12 PRINT
+True block - Correct
+Ln:17 IF
+Ln:17 ELSE executed
+Ln:20 PRINT
+False block - Correct
+Ln:23 IF
+Ln:23 ELSE executed
+Ln:30 PRINT
+False block - Correct
+Ln:33 IF
+Ln:33 ELSE IF executed
+Ln:38 PRINT
+True block - Correct
+Ln:43 PRINT
+IS NOT NULL AND BETWEEN
+Ln:44 IF
+Ln:44 IF TRUE executed
+Ln:45 PRINT
+True block - Correct
+Ln:50 PRINT
+Transact-SQL - Single statement
+Ln:52 IF
+Ln:52 IF TRUE executed
+Ln:53 PRINT
+True block - Correct
+Ln:57 PRINT
+Transact-SQL - BEGIN-END block
+Ln:59 IF
+Ln:59 IF TRUE executed
+Ln:61 PRINT
+True block - Correct
+Ln:62 PRINT
+True block - Correct
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/instr.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/instr.out.txt b/hplsql/src/test/results/local/instr.out.txt
new file mode 100644
index 0000000..a1ed71f
--- /dev/null
+++ b/hplsql/src/test/results/local/instr.out.txt
@@ -0,0 +1,33 @@
+Ln:1 IF
+Ln:1 IF TRUE executed
+Ln:2 PRINT
+Correct
+Ln:7 IF
+Ln:7 IF TRUE executed
+Ln:8 PRINT
+Correct
+Ln:13 IF
+Ln:13 IF TRUE executed
+Ln:14 PRINT
+Correct
+Ln:19 IF
+Ln:19 IF TRUE executed
+Ln:20 PRINT
+Correct
+Ln:25 IF
+Ln:25 IF TRUE executed
+Ln:26 PRINT
+Correct
+Ln:31 DECLARE c STRING
+Ln:33 IF
+Ln:33 IF TRUE executed
+Ln:34 PRINT
+Correct
+Ln:39 IF
+Ln:39 IF TRUE executed
+Ln:40 PRINT
+Correct
+Ln:45 IF
+Ln:45 IF TRUE executed
+Ln:46 PRINT
+Correct
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/interval.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/interval.out.txt b/hplsql/src/test/results/local/interval.out.txt
new file mode 100644
index 0000000..2dcdcd5
--- /dev/null
+++ b/hplsql/src/test/results/local/interval.out.txt
@@ -0,0 +1,11 @@
+2015-03-13
+2015-03-13 00:00:00
+2015-03-12 10:10:10.001
+2015-03-15
+2015-03-15 00:00:00
+2015-03-11
+2015-03-11 00:00:00
+2015-03-12 10:10:09.999
+2015-03-09
+2015-03-09 00:00:00
+2015-03-10 23:59:59

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/lang.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/lang.out.txt b/hplsql/src/test/results/local/lang.out.txt
new file mode 100644
index 0000000..0047ec4
--- /dev/null
+++ b/hplsql/src/test/results/local/lang.out.txt
@@ -0,0 +1,34 @@
+1
+1
+0
+-1
+1.0
+1.0
+-1.0
+Ln:19 DECLARE abc int
+Ln:20 DECLARE abc.abc int
+Ln:21 DECLARE abc . abc1 int
+Ln:22 DECLARE "abc" int
+Ln:23 DECLARE "abc".abc int
+Ln:24 DECLARE "abc"."abc" int
+Ln:25 DECLARE "abc" . "abc1" int
+Ln:26 DECLARE [abc] int
+Ln:27 DECLARE [abc].abc int
+Ln:28 DECLARE [abc].[abc] int
+Ln:29 DECLARE [abc] . [abc1] int
+Ln:30 DECLARE `abc` int
+Ln:31 DECLARE `abc`.abc int
+Ln:32 DECLARE `abc`.`abc` int
+Ln:33 DECLARE `abc` . `abc1` int
+Ln:34 DECLARE :new.abc int
+Ln:35 DECLARE @abc int
+Ln:36 DECLARE _abc int
+Ln:37 DECLARE #abc int
+Ln:38 DECLARE ##abc int
+Ln:39 DECLARE $abc int
+Ln:40 DECLARE abc_9 int
+2
+2
+0
+-2
+0

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/leave.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/leave.out.txt b/hplsql/src/test/results/local/leave.out.txt
new file mode 100644
index 0000000..8e57245
--- /dev/null
+++ b/hplsql/src/test/results/local/leave.out.txt
@@ -0,0 +1,42 @@
+Ln:1 DECLARE count INT = 3
+Ln:3 WHILE - ENTERED
+Ln:4 PRINT
+Start of while block
+Ln:5 PRINT
+3
+Ln:6 SET count = 2
+Ln:7 IF
+Ln:10 PRINT
+End of while block
+Ln:4 PRINT
+Start of while block
+Ln:5 PRINT
+2
+Ln:6 SET count = 1
+Ln:7 IF
+Ln:10 PRINT
+End of while block
+Ln:4 PRINT
+Start of while block
+Ln:5 PRINT
+1
+Ln:6 SET count = 0
+Ln:7 IF
+Ln:7 IF TRUE executed
+Ln:8 LEAVE
+Ln:3 WHILE - LEFT
+Ln:13 SET count = 3
+Ln:16 WHILE - ENTERED
+Ln:17 PRINT
+Start of outer while block
+Ln:20 WHILE - ENTERED
+Ln:21 PRINT
+Start of 1st inner while block
+Ln:22 LEAVE
+Ln:20 WHILE - LEFT
+Ln:27 WHILE - ENTERED
+Ln:28 PRINT
+Start of 2nd inner while block
+Ln:29 LEAVE
+Ln:27 WHILE - LEFT
+Ln:16 WHILE - LEFT
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/len.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/len.out.txt b/hplsql/src/test/results/local/len.out.txt
new file mode 100644
index 0000000..00750ed
--- /dev/null
+++ b/hplsql/src/test/results/local/len.out.txt
@@ -0,0 +1 @@
+3

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/length.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/length.out.txt b/hplsql/src/test/results/local/length.out.txt
new file mode 100644
index 0000000..b8626c4
--- /dev/null
+++ b/hplsql/src/test/results/local/length.out.txt
@@ -0,0 +1 @@
+4

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/lower.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/lower.out.txt b/hplsql/src/test/results/local/lower.out.txt
new file mode 100644
index 0000000..8baef1b
--- /dev/null
+++ b/hplsql/src/test/results/local/lower.out.txt
@@ -0,0 +1 @@
+abc

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/nvl.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/nvl.out.txt b/hplsql/src/test/results/local/nvl.out.txt
new file mode 100644
index 0000000..a111c85
--- /dev/null
+++ b/hplsql/src/test/results/local/nvl.out.txt
@@ -0,0 +1,4 @@
+First non-null
+First non-null
+First non-null
+First non-null

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/nvl2.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/nvl2.out.txt b/hplsql/src/test/results/local/nvl2.out.txt
new file mode 100644
index 0000000..09acc48
--- /dev/null
+++ b/hplsql/src/test/results/local/nvl2.out.txt
@@ -0,0 +1,2 @@
+Correct
+Correct

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/plhqlexception.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/plhqlexception.out.txt b/hplsql/src/test/results/local/plhqlexception.out.txt
new file mode 100644
index 0000000..439cbbb
--- /dev/null
+++ b/hplsql/src/test/results/local/plhqlexception.out.txt
@@ -0,0 +1,6 @@
+Ln:1 PRINT
+Correct
+Ln:2 WHILE - ENTERED
+Ln:3 FETCH
+Ln:3 Cursor not found: cur
+Ln:2 WHILE - LEFT
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/plhqlexception1.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/plhqlexception1.out.txt b/hplsql/src/test/results/local/plhqlexception1.out.txt
new file mode 100644
index 0000000..9b88f62
--- /dev/null
+++ b/hplsql/src/test/results/local/plhqlexception1.out.txt
@@ -0,0 +1,10 @@
+Ln:1 PRINT
+Correct
+Ln:2 DECLARE HANDLER
+Ln:4 WHILE - ENTERED
+Ln:5 FETCH
+Ln:5 Cursor not found: cur
+Ln:4 WHILE - LEFT
+Ln:2 EXIT HANDLER
+Ln:3 PRINT
+Correct - Exception raised

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/plhqlexception2.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/plhqlexception2.out.txt b/hplsql/src/test/results/local/plhqlexception2.out.txt
new file mode 100644
index 0000000..74de5b2
--- /dev/null
+++ b/hplsql/src/test/results/local/plhqlexception2.out.txt
@@ -0,0 +1,106 @@
+Ln:1 DECLARE cnt INT = 0
+Ln:2 PRINT
+Correct
+Ln:3 DECLARE HANDLER
+Ln:5 WHILE - ENTERED
+Ln:6 FETCH
+Ln:6 Cursor not found: cur
+Ln:3 CONTINUE HANDLER
+Ln:4 PRINT
+Correct - Exception raised
+Ln:7 PRINT
+0
+Ln:8 PRINT
+Correct - exception handled
+Ln:9 SET cnt = 1
+Ln:6 FETCH
+Ln:6 Cursor not found: cur
+Ln:3 CONTINUE HANDLER
+Ln:4 PRINT
+Correct - Exception raised
+Ln:7 PRINT
+1
+Ln:8 PRINT
+Correct - exception handled
+Ln:9 SET cnt = 2
+Ln:6 FETCH
+Ln:6 Cursor not found: cur
+Ln:3 CONTINUE HANDLER
+Ln:4 PRINT
+Correct - Exception raised
+Ln:7 PRINT
+2
+Ln:8 PRINT
+Correct - exception handled
+Ln:9 SET cnt = 3
+Ln:6 FETCH
+Ln:6 Cursor not found: cur
+Ln:3 CONTINUE HANDLER
+Ln:4 PRINT
+Correct - Exception raised
+Ln:7 PRINT
+3
+Ln:8 PRINT
+Correct - exception handled
+Ln:9 SET cnt = 4
+Ln:6 FETCH
+Ln:6 Cursor not found: cur
+Ln:3 CONTINUE HANDLER
+Ln:4 PRINT
+Correct - Exception raised
+Ln:7 PRINT
+4
+Ln:8 PRINT
+Correct - exception handled
+Ln:9 SET cnt = 5
+Ln:6 FETCH
+Ln:6 Cursor not found: cur
+Ln:3 CONTINUE HANDLER
+Ln:4 PRINT
+Correct - Exception raised
+Ln:7 PRINT
+5
+Ln:8 PRINT
+Correct - exception handled
+Ln:9 SET cnt = 6
+Ln:6 FETCH
+Ln:6 Cursor not found: cur
+Ln:3 CONTINUE HANDLER
+Ln:4 PRINT
+Correct - Exception raised
+Ln:7 PRINT
+6
+Ln:8 PRINT
+Correct - exception handled
+Ln:9 SET cnt = 7
+Ln:6 FETCH
+Ln:6 Cursor not found: cur
+Ln:3 CONTINUE HANDLER
+Ln:4 PRINT
+Correct - Exception raised
+Ln:7 PRINT
+7
+Ln:8 PRINT
+Correct - exception handled
+Ln:9 SET cnt = 8
+Ln:6 FETCH
+Ln:6 Cursor not found: cur
+Ln:3 CONTINUE HANDLER
+Ln:4 PRINT
+Correct - Exception raised
+Ln:7 PRINT
+8
+Ln:8 PRINT
+Correct - exception handled
+Ln:9 SET cnt = 9
+Ln:6 FETCH
+Ln:6 Cursor not found: cur
+Ln:3 CONTINUE HANDLER
+Ln:4 PRINT
+Correct - Exception raised
+Ln:7 PRINT
+9
+Ln:8 PRINT
+Correct - exception handled
+Ln:9 SET cnt = 10
+Ln:5 WHILE - LEFT

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/print.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/print.out.txt b/hplsql/src/test/results/local/print.out.txt
new file mode 100644
index 0000000..65a1016
--- /dev/null
+++ b/hplsql/src/test/results/local/print.out.txt
@@ -0,0 +1,6 @@
+Ln:1 PRINT
+1
+Ln:2 PRINT
+abc
+Ln:3 PRINT
+abc
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/return.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/return.out.txt b/hplsql/src/test/results/local/return.out.txt
new file mode 100644
index 0000000..f4d5bc9
--- /dev/null
+++ b/hplsql/src/test/results/local/return.out.txt
@@ -0,0 +1,3 @@
+Ln:1 PRINT
+Before return
+Ln:2 RETURN
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/select_conversion.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/select_conversion.out.txt b/hplsql/src/test/results/local/select_conversion.out.txt
new file mode 100644
index 0000000..602304e
--- /dev/null
+++ b/hplsql/src/test/results/local/select_conversion.out.txt
@@ -0,0 +1,9 @@
+Ln:1 DECLARE v1 STRING = abc
+Ln:3 SELECT
+Ln:3 Statement:
+SELECT CONCAT('a', 'b', 'c'), CONCAT('a', 'b') FROM default.dual
+Ln:3 Not executed - offline mode set
+Ln:5 SELECT
+Ln:5 Statement:
+SELECT 'abc' AS c1, CONCAT('abc', 'abc'), NVL(NVL(CONCAT('abc', NVL(id, 1), id), 1), 1), 'abc', 'abc' AS c4 FROM default.dual t1 WHERE 'abc' = 'abc' AND (NVL(NVL('abc', 1), 1) = 'abc' or 'abc' = 'abc')
+Ln:5 Not executed - offline mode set
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/seterror.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/seterror.out.txt b/hplsql/src/test/results/local/seterror.out.txt
new file mode 100644
index 0000000..3c093cc
--- /dev/null
+++ b/hplsql/src/test/results/local/seterror.out.txt
@@ -0,0 +1,6 @@
+Ln:2 SET plhql.onerror = NULL
+Ln:4 HOST
+Ln:4 HOST Command: abcd
+Ln:8 EXCEPTION HANDLER
+Ln:9 PRINT
+FAILED
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/sub.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/sub.out.txt b/hplsql/src/test/results/local/sub.out.txt
new file mode 100644
index 0000000..c35d146
--- /dev/null
+++ b/hplsql/src/test/results/local/sub.out.txt
@@ -0,0 +1 @@
+2014-12-31
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/substr.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/substr.out.txt b/hplsql/src/test/results/local/substr.out.txt
new file mode 100644
index 0000000..09acc48
--- /dev/null
+++ b/hplsql/src/test/results/local/substr.out.txt
@@ -0,0 +1,2 @@
+Correct
+Correct

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/substring.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/substring.out.txt b/hplsql/src/test/results/local/substring.out.txt
new file mode 100644
index 0000000..820d65a
--- /dev/null
+++ b/hplsql/src/test/results/local/substring.out.txt
@@ -0,0 +1,8 @@
+Correct
+Correct
+Ln:4 FUNC SUBSTRING
+Correct
+Ln:5 FUNC SUBSTRING
+Correct
+
+null
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/timestamp.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/timestamp.out.txt b/hplsql/src/test/results/local/timestamp.out.txt
new file mode 100644
index 0000000..bb58d46
--- /dev/null
+++ b/hplsql/src/test/results/local/timestamp.out.txt
@@ -0,0 +1,4 @@
+2015-03-03 11:39:31.123
+2015-03-03 11:39:31.123
+2015-03-03 11:39:31
+2015-03-03 11:39:31.123
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/timestamp_iso.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/timestamp_iso.out.txt b/hplsql/src/test/results/local/timestamp_iso.out.txt
new file mode 100644
index 0000000..997df7f
--- /dev/null
+++ b/hplsql/src/test/results/local/timestamp_iso.out.txt
@@ -0,0 +1,2 @@
+2015-03-12 00:00:00
+2015-03-12 00:00:00

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/to_char.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/to_char.out.txt b/hplsql/src/test/results/local/to_char.out.txt
new file mode 100644
index 0000000..22e8cef
--- /dev/null
+++ b/hplsql/src/test/results/local/to_char.out.txt
@@ -0,0 +1 @@
+2015-04-02

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/to_timestamp.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/to_timestamp.out.txt b/hplsql/src/test/results/local/to_timestamp.out.txt
new file mode 100644
index 0000000..1ee7278
--- /dev/null
+++ b/hplsql/src/test/results/local/to_timestamp.out.txt
@@ -0,0 +1,4 @@
+2015-04-02 00:00:00
+2015-04-02 00:00:00
+2015-04-02 00:00:00
+2015-04-02 13:51:31

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/trim.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/trim.out.txt b/hplsql/src/test/results/local/trim.out.txt
new file mode 100644
index 0000000..bbf851d
--- /dev/null
+++ b/hplsql/src/test/results/local/trim.out.txt
@@ -0,0 +1 @@
+#Hello#

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/twopipes.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/twopipes.out.txt b/hplsql/src/test/results/local/twopipes.out.txt
new file mode 100644
index 0000000..f2ba8f8
--- /dev/null
+++ b/hplsql/src/test/results/local/twopipes.out.txt
@@ -0,0 +1 @@
+abc
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/upper.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/upper.out.txt b/hplsql/src/test/results/local/upper.out.txt
new file mode 100644
index 0000000..5da849b
--- /dev/null
+++ b/hplsql/src/test/results/local/upper.out.txt
@@ -0,0 +1 @@
+ABC

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/values_into.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/values_into.out.txt b/hplsql/src/test/results/local/values_into.out.txt
new file mode 100644
index 0000000..d698e88
--- /dev/null
+++ b/hplsql/src/test/results/local/values_into.out.txt
@@ -0,0 +1,11 @@
+Ln:1 VALUES statement
+Ln:1 SET code = A
+Ln:2 VALUES statement
+Ln:2 SET limit = 0
+Ln:2 SET count = 100
+Ln:4 PRINT
+A
+Ln:5 PRINT
+100
+Ln:6 PRINT
+0
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/7338d8e1/hplsql/src/test/results/local/while.out.txt
----------------------------------------------------------------------
diff --git a/hplsql/src/test/results/local/while.out.txt b/hplsql/src/test/results/local/while.out.txt
new file mode 100644
index 0000000..4a3ca0e
--- /dev/null
+++ b/hplsql/src/test/results/local/while.out.txt
@@ -0,0 +1,72 @@
+Ln:1 DECLARE count INT = 7
+Ln:3 WHILE - ENTERED
+Ln:4 PRINT
+7
+Ln:5 SET count = 6
+Ln:4 PRINT
+6
+Ln:5 SET count = 5
+Ln:4 PRINT
+5
+Ln:5 SET count = 4
+Ln:4 PRINT
+4
+Ln:5 SET count = 3
+Ln:4 PRINT
+3
+Ln:5 SET count = 2
+Ln:4 PRINT
+2
+Ln:5 SET count = 1
+Ln:4 PRINT
+1
+Ln:5 SET count = 0
+Ln:3 WHILE - LEFT
+Ln:8 SET count = 7
+Ln:10 WHILE - ENTERED
+Ln:11 PRINT
+7
+Ln:12 SET count = 6
+Ln:11 PRINT
+6
+Ln:12 SET count = 5
+Ln:11 PRINT
+5
+Ln:12 SET count = 4
+Ln:11 PRINT
+4
+Ln:12 SET count = 3
+Ln:11 PRINT
+3
+Ln:12 SET count = 2
+Ln:11 PRINT
+2
+Ln:12 SET count = 1
+Ln:11 PRINT
+1
+Ln:12 SET count = 0
+Ln:10 WHILE - LEFT
+Ln:15 SET count = 7
+Ln:17 WHILE - ENTERED
+Ln:18 PRINT
+7
+Ln:19 SET count = 6
+Ln:18 PRINT
+6
+Ln:19 SET count = 5
+Ln:18 PRINT
+5
+Ln:19 SET count = 4
+Ln:18 PRINT
+4
+Ln:19 SET count = 3
+Ln:18 PRINT
+3
+Ln:19 SET count = 2
+Ln:18 PRINT
+2
+Ln:19 SET count = 1
+Ln:18 PRINT
+1
+Ln:19 SET count = 0
+Ln:17 WHILE - LEFT
\ No newline at end of file


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

Posted by ga...@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/hbase-metastore
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


[50/50] [abbrv] hive git commit: HIVE-11343 Merge branch 'master' into hbase-metastore

Posted by ga...@apache.org.
HIVE-11343 Merge branch 'master' into hbase-metastore

Conflicts:
	metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
	pom.xml


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

Branch: refs/heads/hbase-metastore
Commit: 61db7b80cbefa5761afd904dfef2eb25e759d6f4
Parents: a310524 2240dbd
Author: Alan Gates <ga...@hortonworks.com>
Authored: Wed Jul 22 13:45:28 2015 -0700
Committer: Alan Gates <ga...@hortonworks.com>
Committed: Wed Jul 22 13:45:28 2015 -0700

----------------------------------------------------------------------
 .gitignore                                      |    1 +
 NOTICE                                          |    2 +-
 RELEASE_NOTES.txt                               |  132 +
 accumulo-handler/pom.xml                        |    2 +-
 .../accumulo/AccumuloConnectionParameters.java  |  111 +-
 .../hive/accumulo/AccumuloStorageHandler.java   |   50 +
 .../hive/accumulo/HiveAccumuloHelper.java       |  280 +
 .../mr/HiveAccumuloTableInputFormat.java        |   74 +-
 .../mr/HiveAccumuloTableOutputFormat.java       |   63 +-
 .../TestAccumuloConnectionParameters.java       |   19 +
 .../hive/accumulo/TestHiveAccumuloHelper.java   |   75 +
 .../mr/TestHiveAccumuloTableInputFormat.java    |    8 +-
 .../mr/TestHiveAccumuloTableOutputFormat.java   |    2 +-
 .../positive/accumulo_predicate_pushdown.q.out  |   76 +-
 .../results/positive/accumulo_queries.q.out     |   70 +-
 ant/pom.xml                                     |    2 +-
 beeline/pom.xml                                 |    2 +-
 .../java/org/apache/hive/beeline/BeeLine.java   |    7 +-
 .../org/apache/hive/beeline/BeeLineOpts.java    |   13 +-
 .../java/org/apache/hive/beeline/Commands.java  |   23 +-
 .../apache/hive/beeline/DatabaseConnection.java |    9 +
 beeline/src/main/resources/BeeLine.properties   |    2 +
 bin/ext/hiveserver2.cmd                         |   51 +-
 bin/ext/hplsql.sh                               |   37 +
 bin/hplsql                                      |   25 +
 bin/hplsql.cmd                                  |   58 +
 cli/pom.xml                                     |    2 +-
 common/pom.xml                                  |    2 +-
 .../apache/hadoop/hive/common/FileUtils.java    |    9 +-
 .../hadoop/hive/common/JvmPauseMonitor.java     |   19 +-
 .../hive/common/jsonexplain/tez/Attr.java       |    6 +-
 .../hive/common/jsonexplain/tez/Connection.java |    6 +-
 .../hadoop/hive/common/jsonexplain/tez/Op.java  |   64 +-
 .../hive/common/jsonexplain/tez/Printer.java    |   41 +
 .../hive/common/jsonexplain/tez/Stage.java      |   95 +-
 .../common/jsonexplain/tez/TezJsonParser.java   |   61 +-
 .../hive/common/jsonexplain/tez/Vertex.java     |   75 +-
 .../hive/common/metrics/LegacyMetrics.java      |   81 +-
 .../hive/common/metrics/common/Metrics.java     |   35 +-
 .../common/metrics/common/MetricsConstant.java  |   35 +
 .../common/metrics/common/MetricsFactory.java   |   30 +-
 .../common/metrics/common/MetricsVariable.java  |   26 +
 .../metrics/metrics2/CodahaleMetrics.java       |   99 +-
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   40 +-
 .../hive/conf/LoopingByteArrayInputStream.java  |    2 +-
 .../hive/common/metrics/TestLegacyMetrics.java  |    6 +-
 .../metrics/metrics2/TestCodahaleMetrics.java   |   58 +-
 contrib/pom.xml                                 |    2 +-
 .../util/typedbytes/TypedBytesInput.java        |    2 +-
 .../util/typedbytes/TypedBytesOutput.java       |    2 +-
 .../util/typedbytes/TypedBytesRecordInput.java  |    2 +-
 .../util/typedbytes/TypedBytesRecordOutput.java |    2 +-
 .../typedbytes/TypedBytesWritableInput.java     |    2 +-
 .../typedbytes/TypedBytesWritableOutput.java    |    2 +-
 data/conf/spark/standalone/hive-site.xml        |    6 +
 data/conf/spark/yarn-client/hive-site.xml       |    6 +
 data/files/ct_events_clean.txt                  |   76 +
 data/files/emp2.txt                             |   16 +
 data/files/encoding-utf8.txt                    |   12 +
 data/files/encoding_iso-8859-1.txt              |    4 +
 data/files/service_request_clean.txt            |   76 +
 data/files/sortdp.txt                           |   32 +
 data/scripts/q_test_cleanup.sql                 |    2 +-
 dev-support/jenkins-submit-build.sh             |   10 +-
 hbase-handler/pom.xml                           |    2 +-
 .../results/positive/external_table_ppd.q.out   |    1 -
 .../positive/hbase_binary_storage_queries.q.out |    2 -
 .../src/test/results/positive/hbase_stats.q.out |    7 -
 .../test/results/positive/hbase_stats2.q.out    |    7 -
 .../test/results/positive/hbase_stats3.q.out    |   12 -
 .../positive/hbase_stats_empty_partition.q.out  |    2 -
 hcatalog/core/pom.xml                           |    2 +-
 .../SemanticAnalysis/HCatSemanticAnalyzer.java  |    7 +-
 hcatalog/hcatalog-pig-adapter/pom.xml           |    2 +-
 .../hcatalog/pig/TestHCatLoaderEncryption.java  |   18 +-
 hcatalog/pom.xml                                |    2 +-
 hcatalog/server-extensions/pom.xml              |    9 +-
 .../src/test/e2e/templeton/deployers/env.sh     |    5 +-
 hcatalog/streaming/pom.xml                      |    8 +-
 .../streaming/AbstractRecordWriter.java         |    4 +-
 .../streaming/mutate/HiveConfFactory.java       |   63 +
 .../mutate/UgiMetaStoreClientFactory.java       |  102 +
 .../streaming/mutate/client/AcidTable.java      |  112 +
 .../mutate/client/AcidTableSerializer.java      |  100 +
 .../mutate/client/ClientException.java          |   15 +
 .../mutate/client/ConnectionException.java      |   15 +
 .../streaming/mutate/client/MutatorClient.java  |  149 +
 .../mutate/client/MutatorClientBuilder.java     |  115 +
 .../streaming/mutate/client/TableType.java      |   37 +
 .../streaming/mutate/client/Transaction.java    |  114 +
 .../mutate/client/TransactionException.java     |   15 +
 .../mutate/client/lock/HeartbeatFactory.java    |   30 +
 .../mutate/client/lock/HeartbeatTimerTask.java  |   66 +
 .../streaming/mutate/client/lock/Lock.java      |  305 +
 .../mutate/client/lock/LockException.java       |   15 +
 .../mutate/client/lock/LockFailureListener.java |   26 +
 .../mutate/doc-files/system-overview.dot        |   27 +
 .../hive/hcatalog/streaming/mutate/package.html |  520 ++
 .../mutate/worker/BucketIdException.java        |   11 +
 .../mutate/worker/BucketIdResolver.java         |   11 +
 .../mutate/worker/BucketIdResolverImpl.java     |   76 +
 .../mutate/worker/GroupRevisitedException.java  |   11 +
 .../mutate/worker/GroupingValidator.java        |   74 +
 .../mutate/worker/MetaStorePartitionHelper.java |  102 +
 .../streaming/mutate/worker/Mutator.java        |   21 +
 .../mutate/worker/MutatorCoordinator.java       |  278 +
 .../worker/MutatorCoordinatorBuilder.java       |  101 +
 .../streaming/mutate/worker/MutatorFactory.java |   16 +
 .../streaming/mutate/worker/MutatorImpl.java    |   86 +
 .../streaming/mutate/worker/OperationType.java  |    7 +
 .../worker/PartitionCreationException.java      |   15 +
 .../mutate/worker/PartitionHelper.java          |   17 +
 .../mutate/worker/RecordInspector.java          |   11 +
 .../mutate/worker/RecordInspectorImpl.java      |   45 +
 .../mutate/worker/RecordSequenceException.java  |   11 +
 .../mutate/worker/SequenceValidator.java        |   49 +
 .../mutate/worker/WarehousePartitionHelper.java |   69 +
 .../mutate/worker/WorkerException.java          |   15 +
 .../streaming/mutate/ExampleUseCase.java        |   82 +
 .../streaming/mutate/MutableRecord.java         |   50 +
 .../mutate/ReflectiveMutatorFactory.java        |   51 +
 .../streaming/mutate/StreamingAssert.java       |  191 +
 .../streaming/mutate/StreamingTestUtils.java    |  261 +
 .../streaming/mutate/TestMutations.java         |  544 ++
 .../mutate/client/TestAcidTableSerializer.java  |   66 +
 .../mutate/client/TestMutatorClient.java        |  176 +
 .../mutate/client/TestTransaction.java          |   95 +
 .../client/lock/TestHeartbeatTimerTask.java     |  100 +
 .../streaming/mutate/client/lock/TestLock.java  |  310 +
 .../mutate/worker/TestBucketIdResolverImpl.java |   38 +
 .../mutate/worker/TestGroupingValidator.java    |   70 +
 .../worker/TestMetaStorePartitionHelper.java    |  112 +
 .../mutate/worker/TestMutatorCoordinator.java   |  244 +
 .../mutate/worker/TestMutatorImpl.java          |   99 +
 .../mutate/worker/TestRecordInspectorImpl.java  |   31 +
 .../mutate/worker/TestSequenceValidator.java    |   91 +
 .../worker/TestWarehousePartitionHelper.java    |   57 +
 hcatalog/webhcat/java-client/pom.xml            |    2 +-
 .../hive/hcatalog/api/HCatClientHMSImpl.java    |   14 +-
 hcatalog/webhcat/svr/pom.xml                    |    2 +-
 .../hive/hcatalog/templeton/AppConfig.java      |   30 +-
 .../hive/hcatalog/templeton/HiveDelegator.java  |   15 +-
 .../hive/hcatalog/templeton/JarDelegator.java   |    8 +-
 .../hcatalog/templeton/LauncherDelegator.java   |   14 +
 .../hive/hcatalog/templeton/PigDelegator.java   |   13 +-
 .../hcatalog/templeton/SecureProxySupport.java  |    6 +-
 .../apache/hive/hcatalog/templeton/Server.java  |   34 +-
 .../hive/hcatalog/templeton/SqoopDelegator.java |   20 +-
 .../hcatalog/templeton/StreamingDelegator.java  |    3 +-
 .../templeton/tool/JobSubmissionConstants.java  |    2 +
 .../hcatalog/templeton/tool/LaunchMapper.java   |  214 +-
 .../templeton/tool/TempletonControllerJob.java  |    7 +-
 hplsql/pom.xml                                  |  128 +
 .../antlr4/org/apache/hive/hplsql/Hplsql.g4     | 1452 ++++
 .../java/org/apache/hive/hplsql/Arguments.java  |  206 +
 .../main/java/org/apache/hive/hplsql/Conf.java  |  175 +
 .../main/java/org/apache/hive/hplsql/Conn.java  |  245 +
 .../java/org/apache/hive/hplsql/Converter.java  |   56 +
 .../main/java/org/apache/hive/hplsql/Copy.java  |  426 ++
 .../main/java/org/apache/hive/hplsql/Exec.java  | 2044 ++++++
 .../java/org/apache/hive/hplsql/Expression.java |  574 ++
 .../main/java/org/apache/hive/hplsql/File.java  |  132 +
 .../java/org/apache/hive/hplsql/Handler.java    |   41 +
 .../java/org/apache/hive/hplsql/Hplsql.java     |   25 +
 .../java/org/apache/hive/hplsql/Interval.java   |  109 +
 .../main/java/org/apache/hive/hplsql/Query.java |  171 +
 .../main/java/org/apache/hive/hplsql/Scope.java |   69 +
 .../java/org/apache/hive/hplsql/Select.java     |  411 ++
 .../java/org/apache/hive/hplsql/Signal.java     |   48 +
 .../main/java/org/apache/hive/hplsql/Stmt.java  | 1082 +++
 .../org/apache/hive/hplsql/StreamGobbler.java   |   51 +
 .../main/java/org/apache/hive/hplsql/Timer.java |   59 +
 .../main/java/org/apache/hive/hplsql/Udf.java   |  117 +
 .../main/java/org/apache/hive/hplsql/Utils.java |  296 +
 .../main/java/org/apache/hive/hplsql/Var.java   |  437 ++
 .../apache/hive/hplsql/functions/Function.java  |  717 ++
 .../hive/hplsql/functions/FunctionDatetime.java |  151 +
 .../hive/hplsql/functions/FunctionMisc.java     |  188 +
 .../hive/hplsql/functions/FunctionOra.java      |  231 +
 .../hive/hplsql/functions/FunctionString.java   |  276 +
 hplsql/src/main/resources/hplsql-site.xml       |   95 +
 .../org/apache/hive/hplsql/TestHplsqlLocal.java |  306 +
 .../db/create_procedure_return_cursor.sql       |   53 +
 .../db/create_procedure_return_cursor2.sql      |   59 +
 hplsql/src/test/queries/local/add.sql           |    2 +
 hplsql/src/test/queries/local/assign.sql        |    7 +
 hplsql/src/test/queries/local/bool_expr.sql     |   47 +
 hplsql/src/test/queries/local/break.sql         |   10 +
 hplsql/src/test/queries/local/case.sql          |   35 +
 hplsql/src/test/queries/local/cast.sql          |    4 +
 hplsql/src/test/queries/local/char.sql          |    1 +
 hplsql/src/test/queries/local/coalesce.sql      |    4 +
 hplsql/src/test/queries/local/concat.sql        |    2 +
 .../src/test/queries/local/create_function.sql  |   11 +
 .../src/test/queries/local/create_function2.sql |   11 +
 .../src/test/queries/local/create_procedure.sql |    9 +
 hplsql/src/test/queries/local/date.sql          |    5 +
 hplsql/src/test/queries/local/dbms_output.sql   |    6 +
 hplsql/src/test/queries/local/declare.sql       |   16 +
 .../test/queries/local/declare_condition.sql    |    8 +
 .../test/queries/local/declare_condition2.sql   |   10 +
 hplsql/src/test/queries/local/decode.sql        |   10 +
 hplsql/src/test/queries/local/equal.sql         |   55 +
 hplsql/src/test/queries/local/exception.sql     |   14 +
 hplsql/src/test/queries/local/exit.sql          |   31 +
 hplsql/src/test/queries/local/expr.sql          |   21 +
 hplsql/src/test/queries/local/for_range.sql     |   20 +
 hplsql/src/test/queries/local/if.sql            |   68 +
 hplsql/src/test/queries/local/instr.sql         |   49 +
 hplsql/src/test/queries/local/interval.sql      |   15 +
 hplsql/src/test/queries/local/lang.sql          |   57 +
 hplsql/src/test/queries/local/leave.sql         |   33 +
 hplsql/src/test/queries/local/len.sql           |    1 +
 hplsql/src/test/queries/local/length.sql        |    1 +
 hplsql/src/test/queries/local/lower.sql         |    1 +
 hplsql/src/test/queries/local/nvl.sql           |    4 +
 hplsql/src/test/queries/local/nvl2.sql          |    2 +
 hplsql/src/test/queries/local/print.sql         |    5 +
 hplsql/src/test/queries/local/return.sql        |    3 +
 hplsql/src/test/queries/local/seterror.sql      |   10 +
 hplsql/src/test/queries/local/sub.sql           |    1 +
 hplsql/src/test/queries/local/substr.sql        |    2 +
 hplsql/src/test/queries/local/substring.sql     |    8 +
 hplsql/src/test/queries/local/timestamp.sql     |    4 +
 hplsql/src/test/queries/local/timestamp_iso.sql |    2 +
 hplsql/src/test/queries/local/to_char.sql       |    1 +
 hplsql/src/test/queries/local/to_timestamp.sql  |    5 +
 hplsql/src/test/queries/local/trim.sql          |    1 +
 hplsql/src/test/queries/local/twopipes.sql      |    1 +
 hplsql/src/test/queries/local/upper.sql         |    1 +
 hplsql/src/test/queries/local/values_into.sql   |    6 +
 hplsql/src/test/queries/local/while.sql         |   20 +
 .../db/create_procedure_return_cursor.out.txt   |  135 +
 .../db/create_procedure_return_cursor2.out.txt  |  139 +
 hplsql/src/test/results/local/add.out.txt       |    2 +
 hplsql/src/test/results/local/assign.out.txt    |    8 +
 hplsql/src/test/results/local/bool_expr.out.txt |   32 +
 hplsql/src/test/results/local/break.out.txt     |   29 +
 hplsql/src/test/results/local/case.out.txt      |   12 +
 hplsql/src/test/results/local/cast.out.txt      |    8 +
 hplsql/src/test/results/local/char.out.txt      |    1 +
 hplsql/src/test/results/local/coalesce.out.txt  |    4 +
 hplsql/src/test/results/local/concat.out.txt    |    2 +
 .../test/results/local/create_function.out.txt  |    9 +
 .../test/results/local/create_function2.out.txt |   10 +
 .../test/results/local/create_procedure.out.txt |    8 +
 hplsql/src/test/results/local/date.out.txt      |    4 +
 .../src/test/results/local/dbms_output.out.txt  |    3 +
 hplsql/src/test/results/local/declare.out.txt   |   13 +
 .../results/local/declare_condition.out.txt     |    7 +
 .../results/local/declare_condition2.out.txt    |   12 +
 hplsql/src/test/results/local/decode.out.txt    |   13 +
 hplsql/src/test/results/local/equal.out.txt     |   48 +
 hplsql/src/test/results/local/exception.out.txt |   13 +
 .../src/test/results/local/exception2.out.txt   |    5 +
 hplsql/src/test/results/local/exit.out.txt      |   42 +
 hplsql/src/test/results/local/expr.out.txt      |   29 +
 hplsql/src/test/results/local/for_range.out.txt |   65 +
 hplsql/src/test/results/local/if.out.txt        |   40 +
 hplsql/src/test/results/local/instr.out.txt     |   33 +
 hplsql/src/test/results/local/interval.out.txt  |   11 +
 hplsql/src/test/results/local/lang.out.txt      |   34 +
 hplsql/src/test/results/local/leave.out.txt     |   42 +
 hplsql/src/test/results/local/len.out.txt       |    1 +
 hplsql/src/test/results/local/length.out.txt    |    1 +
 hplsql/src/test/results/local/lower.out.txt     |    1 +
 hplsql/src/test/results/local/nvl.out.txt       |    4 +
 hplsql/src/test/results/local/nvl2.out.txt      |    2 +
 .../test/results/local/plhqlexception.out.txt   |    6 +
 .../test/results/local/plhqlexception1.out.txt  |   10 +
 .../test/results/local/plhqlexception2.out.txt  |  106 +
 hplsql/src/test/results/local/print.out.txt     |    6 +
 hplsql/src/test/results/local/return.out.txt    |    3 +
 .../results/local/select_conversion.out.txt     |    9 +
 hplsql/src/test/results/local/seterror.out.txt  |    6 +
 hplsql/src/test/results/local/sub.out.txt       |    1 +
 hplsql/src/test/results/local/substr.out.txt    |    2 +
 hplsql/src/test/results/local/substring.out.txt |    8 +
 hplsql/src/test/results/local/timestamp.out.txt |    4 +
 .../test/results/local/timestamp_iso.out.txt    |    2 +
 hplsql/src/test/results/local/to_char.out.txt   |    1 +
 .../src/test/results/local/to_timestamp.out.txt |    4 +
 hplsql/src/test/results/local/trim.out.txt      |    1 +
 hplsql/src/test/results/local/twopipes.out.txt  |    1 +
 hplsql/src/test/results/local/upper.out.txt     |    1 +
 .../src/test/results/local/values_into.out.txt  |   11 +
 hplsql/src/test/results/local/while.out.txt     |   72 +
 hwi/pom.xml                                     |    2 +-
 itests/custom-serde/pom.xml                     |    2 +-
 itests/hcatalog-unit/pom.xml                    |    2 +-
 itests/hive-jmh/pom.xml                         |    2 +-
 itests/hive-minikdc/pom.xml                     |   22 +-
 itests/hive-unit-hadoop2/pom.xml                |    2 +-
 itests/hive-unit/pom.xml                        |    2 +-
 .../hive/metastore/TestHiveMetaStore.java       |   57 +-
 .../hive/metastore/TestMetaStoreMetrics.java    |   66 +-
 .../hive/ql/txn/compactor/TestCompactor.java    |   11 +-
 .../hive/beeline/TestBeeLineWithArgs.java       |   87 +
 .../org/apache/hive/jdbc/TestJdbcDriver2.java   |   16 +
 itests/pom.xml                                  |    3 +-
 itests/qtest-accumulo/pom.xml                   |  508 ++
 .../test/java/org/apache/hive/TestDummy.java    |   31 +
 itests/qtest-spark/pom.xml                      |    2 +-
 itests/qtest/pom.xml                            |   43 +-
 .../test/resources/testconfiguration.properties |   15 +-
 itests/test-serde/pom.xml                       |    2 +-
 itests/util/pom.xml                             |    2 +-
 jdbc/pom.xml                                    |    2 +-
 .../org/apache/hive/jdbc/HiveConnection.java    |   19 +-
 .../apache/hive/jdbc/HivePreparedStatement.java |    2 +-
 metastore/pom.xml                               |    2 +-
 .../upgrade/derby/hive-schema-2.0.0.derby.sql   |  336 +
 .../derby/upgrade-1.2.0-to-2.0.0.derby.sql      |    3 +
 .../scripts/upgrade/derby/upgrade.order.derby   |    2 +-
 .../upgrade/mssql/hive-schema-2.0.0.mssql.sql   |  947 +++
 .../mssql/upgrade-1.2.0-to-2.0.0.mssql.sql      |    5 +
 .../scripts/upgrade/mssql/upgrade.order.mssql   |    2 +-
 .../upgrade/mysql/021-HIVE-7018.mysql.sql       |   53 +
 .../upgrade/mysql/hive-schema-1.3.0.mysql.sql   |   10 +-
 .../upgrade/mysql/hive-schema-2.0.0.mysql.sql   |  834 +++
 .../mysql/upgrade-1.2.0-to-1.3.0.mysql.sql      |    2 +-
 .../mysql/upgrade-1.2.0-to-2.0.0.mysql.sql      |    4 +
 .../scripts/upgrade/mysql/upgrade.order.mysql   |    2 +-
 .../upgrade/oracle/hive-schema-2.0.0.oracle.sql |  788 ++
 .../oracle/upgrade-1.2.0-to-2.0.0.oracle.sql    |    4 +
 .../scripts/upgrade/oracle/upgrade.order.oracle |    2 +-
 .../postgres/hive-schema-2.0.0.postgres.sql     | 1457 ++++
 .../upgrade-1.2.0-to-2.0.0.postgres.sql         |   12 +
 .../upgrade/postgres/upgrade.order.postgres     |    2 +-
 .../hive/metastore/AggregateStatsCache.java     |    5 +-
 .../apache/hadoop/hive/metastore/Deadline.java  |    2 +-
 .../hadoop/hive/metastore/HiveMetaStore.java    |  241 +-
 .../hive/metastore/HiveMetaStoreClient.java     |   20 +-
 .../hadoop/hive/metastore/IMetaStoreClient.java |   61 +-
 .../hive/metastore/MetaStoreDirectSql.java      |   72 +-
 .../hive/metastore/MetaStoreSchemaInfo.java     |   14 +-
 .../hadoop/hive/metastore/MetaStoreUtils.java   |   26 -
 .../hadoop/hive/metastore/ObjectStore.java      | 1885 +++--
 .../hive/metastore/PartitionDropOptions.java    |    6 -
 .../hadoop/hive/metastore/ProtectMode.java      |   97 -
 .../hive/metastore/StatObjectConverter.java     |   16 +-
 .../hive/metastore/tools/HiveMetaTool.java      |   23 +-
 .../hive/metastore/txn/CompactionInfo.java      |    9 +
 .../metastore/txn/CompactionTxnHandler.java     |   19 +-
 .../hadoop/hive/metastore/txn/TxnHandler.java   |   36 +-
 .../hadoop/hive/metastore/TestObjectStore.java  |  230 +
 odbc/pom.xml                                    |    2 +-
 packaging/pom.xml                               |    2 +-
 pom.xml                                         |   32 +-
 ql/pom.xml                                      |   14 +-
 .../UDAFTemplates/VectorUDAFMinMaxString.txt    |    3 +-
 .../java/org/apache/hadoop/hive/ql/Driver.java  |   43 +-
 .../org/apache/hadoop/hive/ql/ErrorMsg.java     |   22 +-
 .../org/apache/hadoop/hive/ql/QueryPlan.java    |    9 +-
 .../apache/hadoop/hive/ql/exec/ColumnInfo.java  |    2 +-
 .../hive/ql/exec/CommonMergeJoinOperator.java   |   76 +-
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |  207 +-
 .../apache/hadoop/hive/ql/exec/ExplainTask.java |   55 +-
 .../hadoop/hive/ql/exec/FetchOperator.java      |    5 +-
 .../hadoop/hive/ql/exec/FilterOperator.java     |    3 +-
 .../apache/hadoop/hive/ql/exec/JoinUtil.java    |   87 +-
 .../hadoop/hive/ql/exec/MapJoinOperator.java    |   61 +-
 .../apache/hadoop/hive/ql/exec/Operator.java    |    6 +
 .../hive/ql/exec/OrcFileMergeOperator.java      |   70 +-
 .../hive/ql/exec/PartitionKeySampler.java       |    9 +-
 .../hadoop/hive/ql/exec/ReduceSinkOperator.java |    7 +-
 .../apache/hadoop/hive/ql/exec/TaskFactory.java |    2 +-
 .../apache/hadoop/hive/ql/exec/Utilities.java   |   30 +-
 .../hadoop/hive/ql/exec/mr/ExecDriver.java      |   12 +-
 .../hive/ql/exec/mr/ExecMapperContext.java      |   10 +-
 .../persistence/HybridHashTableContainer.java   |   59 +-
 .../persistence/MapJoinBytesTableContainer.java |   75 +-
 .../exec/persistence/MapJoinTableContainer.java |    5 +
 .../ql/exec/spark/SparkReduceRecordHandler.java |   31 +-
 .../hive/ql/exec/tez/HashTableLoader.java       |   81 +-
 .../hive/ql/exec/tez/HiveSplitGenerator.java    |   24 +
 .../hive/ql/exec/tez/KeyValuesAdapter.java      |   47 +
 .../hive/ql/exec/tez/KeyValuesFromKeyValue.java |   90 +
 .../ql/exec/tez/KeyValuesFromKeyValues.java     |   48 +
 .../ql/exec/tez/MergeFileRecordProcessor.java   |   42 +-
 .../hive/ql/exec/tez/ReduceRecordProcessor.java |   11 +-
 .../hive/ql/exec/tez/ReduceRecordSource.java    |   48 +-
 .../hadoop/hive/ql/exec/tez/TezJobMonitor.java  |    2 +-
 .../hive/ql/exec/tez/TezSessionPoolManager.java |   21 +-
 .../hive/ql/exec/tez/TezSessionState.java       |   10 -
 .../apache/hadoop/hive/ql/exec/tez/TezTask.java |   10 +-
 .../hive/ql/exec/vector/BytesColumnVector.java  |   14 +
 .../hive/ql/exec/vector/ColumnVector.java       |    8 +
 .../ql/exec/vector/DecimalColumnVector.java     |   12 +
 .../hive/ql/exec/vector/DoubleColumnVector.java |   12 +
 .../hive/ql/exec/vector/LongColumnVector.java   |   12 +
 .../ql/exec/vector/VectorGroupByOperator.java   |   16 +-
 .../ql/exec/vector/VectorMapJoinOperator.java   |    1 -
 .../ql/exec/vector/VectorSelectOperator.java    |    3 -
 .../ql/exec/vector/VectorizationContext.java    |   14 +-
 .../ql/exec/vector/VectorizedBatchUtil.java     |   41 +-
 .../hive/ql/exec/vector/VectorizedRowBatch.java |   78 +-
 .../ql/exec/vector/VectorizedRowBatchCtx.java   |    2 +-
 .../mapjoin/VectorMapJoinCommonOperator.java    |    1 +
 .../mapjoin/VectorMapJoinRowBytesContainer.java |    9 +-
 .../fast/VectorMapJoinFastHashTable.java        |    5 +
 .../fast/VectorMapJoinFastTableContainer.java   |    5 +
 .../hashtable/VectorMapJoinHashTable.java       |    4 +
 .../VectorMapJoinOptimizedHashTable.java        |    4 +
 .../apache/hadoop/hive/ql/hooks/ATSHook.java    |   27 +-
 .../hadoop/hive/ql/hooks/HookContext.java       |   12 +
 .../hadoop/hive/ql/hooks/LineageInfo.java       |   96 +
 .../hadoop/hive/ql/hooks/LineageLogger.java     |  439 ++
 .../hadoop/hive/ql/hooks/WriteEntity.java       |    6 +-
 .../hadoop/hive/ql/io/AcidInputFormat.java      |   60 +-
 .../hadoop/hive/ql/io/AcidOutputFormat.java     |   49 +-
 .../org/apache/hadoop/hive/ql/io/AcidUtils.java |  152 +-
 .../hadoop/hive/ql/io/FileFormatException.java  |   30 +
 .../ql/io/HiveContextAwareRecordReader.java     |    2 +-
 .../hadoop/hive/ql/io/HiveFileFormatUtils.java  |   19 +-
 .../hadoop/hive/ql/io/HiveInputFormat.java      |   38 +-
 .../org/apache/hadoop/hive/ql/io/IOContext.java |   43 -
 .../apache/hadoop/hive/ql/io/IOContextMap.java  |   81 +
 .../hive/ql/io/orc/ColumnStatisticsImpl.java    |  217 +-
 .../apache/hadoop/hive/ql/io/orc/FileDump.java  |    7 +-
 .../hadoop/hive/ql/io/orc/MemoryManager.java    |   43 +-
 .../apache/hadoop/hive/ql/io/orc/OrcConf.java   |  134 +
 .../apache/hadoop/hive/ql/io/orc/OrcFile.java   |   55 +-
 .../hive/ql/io/orc/OrcFileKeyWrapper.java       |   27 +-
 .../io/orc/OrcFileStripeMergeRecordReader.java  |   23 +-
 .../hadoop/hive/ql/io/orc/OrcInputFormat.java   |   60 +-
 .../hive/ql/io/orc/OrcNewInputFormat.java       |   16 +-
 .../hadoop/hive/ql/io/orc/OrcNewSplit.java      |   13 +-
 .../hive/ql/io/orc/OrcRawRecordMerger.java      |   84 +-
 .../hadoop/hive/ql/io/orc/OrcRecordUpdater.java |   63 +-
 .../apache/hadoop/hive/ql/io/orc/OrcSplit.java  |   16 +-
 .../apache/hadoop/hive/ql/io/orc/OrcUtils.java  |    4 +
 .../apache/hadoop/hive/ql/io/orc/Reader.java    |   34 +-
 .../hadoop/hive/ql/io/orc/ReaderImpl.java       |   19 +-
 .../hive/ql/io/orc/RecordReaderFactory.java     |    7 +-
 .../hadoop/hive/ql/io/orc/RecordReaderImpl.java |   42 +-
 .../ql/io/orc/RunLengthIntegerReaderV2.java     |    8 +-
 .../hadoop/hive/ql/io/orc/WriterImpl.java       |   96 +-
 .../io/parquet/FilterPredicateLeafBuilder.java  |    8 +-
 .../hive/ql/io/parquet/LeafFilterFactory.java   |   20 +-
 .../ql/io/parquet/MapredParquetInputFormat.java |    3 +-
 .../io/parquet/MapredParquetOutputFormat.java   |    2 +-
 .../parquet/VectorizedParquetInputFormat.java   |    2 +-
 .../convert/DataWritableRecordConverter.java    |    8 +-
 .../ql/io/parquet/convert/ETypeConverter.java   |   10 +-
 .../convert/HiveCollectionConverter.java        |    6 +-
 .../io/parquet/convert/HiveGroupConverter.java  |   14 +-
 .../io/parquet/convert/HiveSchemaConverter.java |   18 +-
 .../io/parquet/convert/HiveStructConverter.java |    6 +-
 .../hive/ql/io/parquet/convert/Repeated.java    |   12 +-
 .../parquet/read/DataWritableReadSupport.java   |   20 +-
 .../read/ParquetRecordReaderWrapper.java        |  133 +-
 .../ql/io/parquet/serde/ParquetHiveSerDe.java   |    4 +-
 .../hive/ql/io/parquet/timestamp/NanoTime.java  |    6 +-
 .../parquet/write/DataWritableWriteSupport.java |    8 +-
 .../ql/io/parquet/write/DataWritableWriter.java |  648 +-
 .../write/ParquetRecordWriterWrapper.java       |    6 +-
 .../hive/ql/io/sarg/SearchArgumentImpl.java     |  343 +-
 .../hadoop/hive/ql/lib/RuleExactMatch.java      |   21 +-
 .../apache/hadoop/hive/ql/lib/RuleRegExp.java   |  191 +-
 .../hadoop/hive/ql/lockmgr/DbLockManager.java   |    4 +-
 .../hadoop/hive/ql/lockmgr/DbTxnManager.java    |   22 +-
 .../hadoop/hive/ql/lockmgr/DummyTxnManager.java |   19 +-
 .../hadoop/hive/ql/lockmgr/HiveLockObject.java  |   37 +-
 .../hadoop/hive/ql/lockmgr/HiveTxnManager.java  |   50 +-
 .../hive/ql/lockmgr/HiveTxnManagerImpl.java     |  140 +
 .../zookeeper/ZooKeeperHiveLockManager.java     |    2 +
 .../ql/log/NoDeleteRollingFileAppender.java     |  176 +
 .../apache/hadoop/hive/ql/log/PerfLogger.java   |    1 -
 .../apache/hadoop/hive/ql/metadata/Hive.java    |   77 +-
 .../hadoop/hive/ql/metadata/HiveUtils.java      |    8 -
 .../hadoop/hive/ql/metadata/Partition.java      |   51 -
 .../apache/hadoop/hive/ql/metadata/Table.java   |   65 +-
 .../formatting/MetaDataFormatUtils.java         |   24 +-
 .../BucketingSortingReduceSinkOptimizer.java    |    4 +
 .../hive/ql/optimizer/ColumnPrunerProcCtx.java  |   27 +
 .../ql/optimizer/ColumnPrunerProcFactory.java   |   49 +-
 .../hive/ql/optimizer/ConstantPropagate.java    |   13 +-
 .../ql/optimizer/ConstantPropagateProcCtx.java  |   31 +-
 .../optimizer/ConstantPropagateProcFactory.java |  169 +-
 .../hive/ql/optimizer/ConvertJoinMapJoin.java   |  234 +-
 .../hive/ql/optimizer/MapJoinProcessor.java     |   44 +-
 .../ql/optimizer/NonBlockingOpDeDupProc.java    |    9 -
 .../hadoop/hive/ql/optimizer/Optimizer.java     |    4 +
 .../hadoop/hive/ql/optimizer/PrunerUtils.java   |   14 +-
 .../ql/optimizer/ReduceSinkMapJoinProc.java     |   84 +-
 .../optimizer/RemoveDynamicPruningBySize.java   |    2 +-
 .../ql/optimizer/calcite/HiveCalciteUtil.java   |  148 +-
 .../ql/optimizer/calcite/HiveRelOptUtil.java    |  316 +
 .../calcite/cost/HiveOnTezCostModel.java        |   25 +-
 .../calcite/reloperators/HiveJoin.java          |   25 +-
 .../calcite/reloperators/HiveMultiJoin.java     |  221 +
 .../calcite/reloperators/HiveSemiJoin.java      |   57 +-
 .../rules/HiveInsertExchange4JoinRule.java      |   40 +-
 .../calcite/rules/HiveJoinAddNotNullRule.java   |   16 +-
 .../calcite/rules/HiveJoinCommuteRule.java      |   96 +
 .../rules/HiveJoinProjectTransposeRule.java     |  283 +
 .../HiveJoinPushTransitivePredicatesRule.java   |  139 +
 .../calcite/rules/HiveJoinToMultiJoinRule.java  |  387 +-
 .../calcite/rules/HiveProjectMergeRule.java     |    1 -
 .../calcite/rules/HiveRelFieldTrimmer.java      |  107 +
 .../calcite/stats/HiveRelMdSelectivity.java     |   11 +-
 .../calcite/translator/ASTConverter.java        |   20 +-
 .../calcite/translator/ExprNodeConverter.java   |   41 +-
 .../calcite/translator/HiveOpConverter.java     |  363 +-
 .../translator/HiveOpConverterPostProc.java     |   56 +-
 .../calcite/translator/JoinTypeCheckCtx.java    |    2 +-
 .../translator/PlanModifierForASTConv.java      |   12 +-
 .../correlation/ReduceSinkDeDuplication.java    |    3 +
 .../ql/optimizer/lineage/ExprProcFactory.java   |   98 +
 .../hive/ql/optimizer/lineage/Generator.java    |   16 +-
 .../hive/ql/optimizer/lineage/LineageCtx.java   |   79 +-
 .../ql/optimizer/lineage/OpProcFactory.java     |  228 +-
 .../BucketingSortingInferenceOptimizer.java     |    8 +-
 .../ql/optimizer/physical/MemoryDecider.java    |  288 +
 .../ql/optimizer/physical/SerializeFilter.java  |  178 +
 .../hive/ql/optimizer/physical/Vectorizer.java  |  147 +-
 .../stats/annotation/StatsRulesProcFactory.java |   42 +-
 .../hadoop/hive/ql/parse/CalcitePlanner.java    |   95 +-
 .../hive/ql/parse/DDLSemanticAnalyzer.java      |  112 +-
 .../apache/hadoop/hive/ql/parse/EximUtil.java   |   31 +-
 .../hive/ql/parse/ExplainSemanticAnalyzer.java  |   11 +-
 .../hadoop/hive/ql/parse/GenTezProcContext.java |   20 +
 .../hadoop/hive/ql/parse/GenTezUtils.java       |   90 +-
 .../apache/hadoop/hive/ql/parse/GenTezWork.java |   91 +-
 .../org/apache/hadoop/hive/ql/parse/HiveLexer.g |    5 -
 .../apache/hadoop/hive/ql/parse/HiveParser.g    |   38 +-
 .../hive/ql/parse/LoadSemanticAnalyzer.java     |   79 +-
 .../hadoop/hive/ql/parse/PTFInvocationSpec.java |    8 +
 .../hadoop/hive/ql/parse/PTFTranslator.java     |    1 +
 .../apache/hadoop/hive/ql/parse/ParseUtils.java |   51 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |  217 +-
 .../hive/ql/parse/SemanticAnalyzerFactory.java  |    8 +-
 .../hadoop/hive/ql/parse/TezCompiler.java       |   33 +-
 .../hadoop/hive/ql/parse/TypeCheckCtx.java      |   15 +-
 .../hive/ql/parse/TypeCheckProcFactory.java     |    4 +
 .../hadoop/hive/ql/parse/WindowingSpec.java     |   84 +-
 .../hive/ql/plan/AbstractOperatorDesc.java      |   14 +
 .../hadoop/hive/ql/plan/AlterTableDesc.java     |   20 +-
 .../apache/hadoop/hive/ql/plan/BaseWork.java    |    9 +-
 .../hive/ql/plan/CommonMergeJoinDesc.java       |    4 +
 .../hadoop/hive/ql/plan/ConditionalWork.java    |    4 +-
 .../hadoop/hive/ql/plan/DropTableDesc.java      |   26 +-
 .../apache/hadoop/hive/ql/plan/ExplainWork.java |   19 +-
 .../hadoop/hive/ql/plan/ExprNodeDescUtils.java  |  115 +
 .../hadoop/hive/ql/plan/FileSinkDesc.java       |   27 +-
 .../apache/hadoop/hive/ql/plan/FilterDesc.java  |   14 +
 .../hadoop/hive/ql/plan/HiveOperation.java      |    2 -
 .../apache/hadoop/hive/ql/plan/JoinDesc.java    |   11 +-
 .../apache/hadoop/hive/ql/plan/MapJoinDesc.java |   11 +
 .../hadoop/hive/ql/plan/MergeJoinWork.java      |    8 +-
 .../hadoop/hive/ql/plan/OperatorDesc.java       |    2 +
 .../apache/hadoop/hive/ql/plan/ReduceWork.java  |    2 +-
 .../hadoop/hive/ql/plan/TableScanDesc.java      |   21 +-
 .../hadoop/hive/ql/plan/VectorGroupByDesc.java  |   23 +-
 .../hive/ql/ppd/ExprWalkerProcFactory.java      |   17 +-
 .../hadoop/hive/ql/processors/SetProcessor.java |    3 +
 .../authorization/plugin/HiveAuthorizer.java    |   11 +
 .../plugin/HiveAuthorizerImpl.java              |   22 +
 .../authorization/plugin/HiveV1Authorizer.java  |   20 +
 .../hadoop/hive/ql/session/LineageState.java    |    9 +-
 .../hadoop/hive/ql/session/OperationLog.java    |    2 +-
 .../hadoop/hive/ql/session/SessionState.java    |   40 +-
 .../apache/hadoop/hive/ql/stats/StatsUtils.java |   14 +-
 .../hive/ql/txn/compactor/CompactorMR.java      |    4 +-
 .../ql/udf/generic/GenericUDAFCollectList.java  |    5 -
 .../ql/udf/generic/GenericUDAFCollectSet.java   |    5 -
 .../ql/udf/generic/GenericUDAFComputeStats.java |    4 +-
 .../hive/ql/udf/generic/GenericUDAFCount.java   |    4 -
 .../ql/udf/generic/GenericUDAFCovariance.java   |    4 -
 .../ql/udf/generic/GenericUDAFCumeDist.java     |    4 -
 .../ql/udf/generic/GenericUDAFDenseRank.java    |    4 -
 .../generic/GenericUDAFHistogramNumeric.java    |    2 +-
 .../generic/GenericUDAFPercentileApprox.java    |    2 +-
 .../hive/ql/udf/generic/GenericUDFMapKeys.java  |    6 +-
 .../udf/generic/NumDistinctValueEstimator.java  |   18 +
 .../hive/ql/udf/generic/NumericHistogram.java   |   14 +
 .../hive/ql/udf/ptf/WindowingTableFunction.java |  135 +-
 .../hadoop/hive/ql/util/JavaDataModel.java      |   30 -
 .../apache/hadoop/hive/ql/TestTxnCommands2.java |   68 +-
 .../hive/ql/exec/TestFileSinkOperator.java      |    3 +-
 .../hadoop/hive/ql/exec/TestOperators.java      |    3 +-
 .../hadoop/hive/ql/exec/TestUtilities.java      |   11 +
 .../exec/vector/TestVectorGroupByOperator.java  |    2 +-
 .../apache/hadoop/hive/ql/io/TestAcidUtils.java |   73 +-
 .../ql/io/TestHiveBinarySearchRecordReader.java |    2 +-
 .../hadoop/hive/ql/io/TestIOContextMap.java     |  133 +
 .../hadoop/hive/ql/io/orc/TestFileDump.java     |   57 +-
 .../hive/ql/io/orc/TestInputOutputFormat.java   |  114 +-
 .../hive/ql/io/orc/TestOrcRawRecordMerger.java  |   57 +-
 .../hive/ql/io/orc/TestOrcRecordUpdater.java    |    6 +-
 .../io/parquet/AbstractTestParquetDirect.java   |    8 +-
 .../ql/io/parquet/TestArrayCompatibility.java   |   18 +-
 .../ql/io/parquet/TestDataWritableWriter.java   |    8 +-
 .../ql/io/parquet/TestHiveSchemaConverter.java  |   16 +-
 .../hive/ql/io/parquet/TestMapStructures.java   |   10 +-
 .../parquet/TestMapredParquetInputFormat.java   |    2 +-
 .../parquet/TestMapredParquetOutputFormat.java  |    2 +-
 .../io/parquet/TestParquetRowGroupFilter.java   |    6 +-
 .../hive/ql/io/sarg/TestSearchArgumentImpl.java |   34 +-
 .../hadoop/hive/ql/lib/TestRuleRegExp.java      |  118 +
 .../hive/ql/lockmgr/TestDummyTxnManager.java    |  119 +-
 .../parse/TestUpdateDeleteSemanticAnalyzer.java |   36 +-
 .../hive/ql/txn/compactor/CompactorTest.java    |   20 +-
 .../hive/ql/txn/compactor/TestCleaner.java      |    8 +-
 .../hive/ql/txn/compactor/TestCleaner2.java     |   14 +
 .../hive/ql/txn/compactor/TestInitiator.java    |    4 +
 .../hive/ql/txn/compactor/TestWorker.java       |   49 +-
 .../hive/ql/txn/compactor/TestWorker2.java      |   16 +
 .../alter_partition_invalidspec.q               |    8 -
 .../clientnegative/alter_partition_nodrop.q     |    9 -
 .../alter_partition_nodrop_table.q              |    9 -
 .../clientnegative/alter_partition_offline.q    |   11 -
 .../clientnegative/drop_table_failure3.q        |   12 -
 .../queries/clientnegative/exchange_partition.q |   19 +
 .../queries/clientnegative/load_orc_negative1.q |    4 +
 .../queries/clientnegative/load_orc_negative2.q |    6 +
 .../clientnegative/load_orc_negative_part.q     |   14 +
 .../clientnegative/lockneg_try_lock_db_in_use.q |    2 +-
 .../queries/clientnegative/protectmode_part.q   |   15 -
 .../queries/clientnegative/protectmode_part1.q  |   21 -
 .../queries/clientnegative/protectmode_part2.q  |    9 -
 .../clientnegative/protectmode_part_no_drop.q   |   10 -
 .../clientnegative/protectmode_part_no_drop2.q  |   11 -
 .../queries/clientnegative/protectmode_tbl1.q   |    8 -
 .../queries/clientnegative/protectmode_tbl2.q   |   12 -
 .../queries/clientnegative/protectmode_tbl3.q   |   10 -
 .../queries/clientnegative/protectmode_tbl4.q   |   15 -
 .../queries/clientnegative/protectmode_tbl5.q   |   15 -
 .../queries/clientnegative/protectmode_tbl6.q   |    8 -
 .../queries/clientnegative/protectmode_tbl7.q   |   13 -
 .../queries/clientnegative/protectmode_tbl8.q   |   13 -
 .../clientnegative/protectmode_tbl_no_drop.q    |    9 -
 .../clientnegative/ptf_negative_NoWindowDefn.q  |    9 +
 .../test/queries/clientnegative/sa_fail_hook3.q |    4 -
 .../alter_partition_protect_mode.q              |   26 -
 .../queries/clientpositive/ambiguitycheck.q     |    2 -
 ql/src/test/queries/clientpositive/auto_join0.q |    1 +
 ql/src/test/queries/clientpositive/auto_join1.q |    1 +
 .../test/queries/clientpositive/auto_join21.q   |    1 +
 .../test/queries/clientpositive/auto_join29.q   |    1 +
 .../test/queries/clientpositive/auto_join30.q   |    1 +
 .../clientpositive/auto_sortmerge_join_10.q     |    1 +
 .../clientpositive/auto_sortmerge_join_13.q     |    1 +
 .../clientpositive/auto_sortmerge_join_14.q     |    1 +
 .../clientpositive/auto_sortmerge_join_15.q     |    1 +
 .../clientpositive/auto_sortmerge_join_9.q      |    1 +
 .../queries/clientpositive/avro_timestamp.q     |    2 +
 .../queries/clientpositive/avro_timestamp_win.q |   28 +
 .../queries/clientpositive/ba_table_union.q     |    2 +-
 ql/src/test/queries/clientpositive/bucket2.q    |    1 +
 ql/src/test/queries/clientpositive/bucket3.q    |    1 +
 ql/src/test/queries/clientpositive/bucket4.q    |    1 +
 .../clientpositive/bucket_map_join_tez1.q       |    1 +
 .../clientpositive/bucket_map_join_tez2.q       |    1 +
 .../queries/clientpositive/cbo_rp_auto_join0.q  |    1 +
 .../queries/clientpositive/cbo_rp_auto_join1.q  |    1 +
 .../test/queries/clientpositive/cbo_rp_join0.q  |   27 +
 .../test/queries/clientpositive/cbo_rp_join1.q  |   22 +
 .../test/queries/clientpositive/cbo_subq_in.q   |    1 +
 .../test/queries/clientpositive/cbo_windowing.q |    4 +-
 .../test/queries/clientpositive/constprog_dpp.q |   17 +
 .../clientpositive/correlationoptimizer1.q      |    1 +
 ql/src/test/queries/clientpositive/count.q      |    1 +
 ql/src/test/queries/clientpositive/cp_sel.q     |   11 +
 ql/src/test/queries/clientpositive/cross_join.q |    1 +
 .../clientpositive/cross_product_check_1.q      |    1 +
 .../clientpositive/cross_product_check_2.q      |    1 +
 ql/src/test/queries/clientpositive/ctas.q       |    1 +
 .../disable_merge_for_bucketing.q               |    1 +
 .../drop_partitions_ignore_protection.q         |   10 -
 .../clientpositive/dynamic_partition_pruning.q  |    1 +
 .../dynamic_partition_pruning_2.q               |    1 +
 .../clientpositive/dynpart_sort_opt_bucketing.q |   73 +
 .../dynpart_sort_opt_vectorization.q            |    1 +
 .../clientpositive/dynpart_sort_optimization.q  |    1 +
 .../clientpositive/dynpart_sort_optimization2.q |    1 +
 .../queries/clientpositive/encoding_nonutf8.q   |    7 +
 .../clientpositive/encryption_drop_partition.q  |   18 +
 .../encryption_insert_partition_static.q        |   17 -
 .../test/queries/clientpositive/explainuser_3.q |  115 +
 .../queries/clientpositive/fouter_join_ppr.q    |   73 +
 ql/src/test/queries/clientpositive/groupby1.q   |    1 +
 ql/src/test/queries/clientpositive/groupby2.q   |    1 +
 ql/src/test/queries/clientpositive/groupby3.q   |    1 +
 ...groupby_complex_types_multi_single_reducer.q |    8 +-
 .../clientpositive/groupby_grouping_sets6.q     |    1 -
 ql/src/test/queries/clientpositive/having.q     |    1 +
 .../clientpositive/hybridgrace_hashjoin_1.q     |    1 +
 .../clientpositive/hybridgrace_hashjoin_2.q     |    6 +-
 .../test/queries/clientpositive/insert_into1.q  |    1 +
 .../test/queries/clientpositive/insert_into2.q  |    1 +
 .../insert_non_utf8_encoding_table.q            |   20 +
 ql/src/test/queries/clientpositive/join0.q      |    1 +
 ql/src/test/queries/clientpositive/join1.q      |    1 +
 ql/src/test/queries/clientpositive/join42.q     |   36 +
 ql/src/test/queries/clientpositive/join43.q     |   83 +
 .../join_merge_multi_expressions.q              |    1 +
 .../test/queries/clientpositive/join_nullsafe.q |    1 +
 .../queries/clientpositive/limit_pushdown.q     |    1 +
 ql/src/test/queries/clientpositive/lineage2.q   |  116 +
 ql/src/test/queries/clientpositive/lineage3.q   |  162 +
 .../queries/clientpositive/load_dyn_part1.q     |    1 +
 .../queries/clientpositive/load_dyn_part2.q     |    1 +
 .../queries/clientpositive/load_dyn_part3.q     |    1 +
 ql/src/test/queries/clientpositive/load_orc.q   |   10 +
 .../test/queries/clientpositive/load_orc_part.q |   15 +
 .../test/queries/clientpositive/lvj_mapjoin.q   |    1 +
 .../queries/clientpositive/mapjoin_decimal.q    |    1 +
 .../queries/clientpositive/mapjoin_mapjoin.q    |    1 +
 ql/src/test/queries/clientpositive/mapreduce1.q |    1 +
 ql/src/test/queries/clientpositive/mapreduce2.q |    1 +
 ql/src/test/queries/clientpositive/merge1.q     |    1 +
 ql/src/test/queries/clientpositive/merge2.q     |    1 +
 ql/src/test/queries/clientpositive/mergejoin.q  |    3 +-
 .../clientpositive/metadata_only_queries.q      |    1 +
 ql/src/test/queries/clientpositive/mrr.q        |    3 +
 ql/src/test/queries/clientpositive/orc_merge1.q |    1 +
 ql/src/test/queries/clientpositive/orc_merge2.q |    1 +
 ql/src/test/queries/clientpositive/orc_merge3.q |    1 +
 ql/src/test/queries/clientpositive/orc_merge4.q |    1 +
 ql/src/test/queries/clientpositive/orc_merge5.q |    1 +
 ql/src/test/queries/clientpositive/orc_merge6.q |    1 +
 ql/src/test/queries/clientpositive/orc_merge7.q |    1 +
 ql/src/test/queries/clientpositive/orc_merge9.q |   44 +
 .../clientpositive/orc_merge_incompat1.q        |    1 +
 .../clientpositive/orc_merge_incompat2.q        |    1 +
 ql/src/test/queries/clientpositive/parallel.q   |    1 +
 .../clientpositive/partition_timestamp.q        |    2 +
 .../clientpositive/partition_timestamp2.q       |    2 +
 .../clientpositive/partition_timestamp2_win.q   |   58 +
 .../clientpositive/partition_timestamp_win.q    |   59 +
 .../test/queries/clientpositive/protectmode.q   |   63 -
 .../test/queries/clientpositive/protectmode2.q  |   23 -
 ql/src/test/queries/clientpositive/ptf.q        |    1 +
 .../test/queries/clientpositive/ptf_matchpath.q |    1 +
 .../test/queries/clientpositive/ptf_streaming.q |    1 +
 .../queries/clientpositive/ql_rewrite_gbtoidx.q |    4 +-
 .../test/queries/clientpositive/script_pipe.q   |    1 +
 .../queries/clientpositive/selectDistinctStar.q |    3 +-
 .../clientpositive/select_dummy_source.q        |    1 +
 .../queries/clientpositive/select_same_col.q    |    5 +-
 ql/src/test/queries/clientpositive/skewjoin.q   |    1 +
 .../clientpositive/smb_join_partition_key.q     |   35 +
 .../queries/clientpositive/stats_noscan_1.q     |    1 +
 .../queries/clientpositive/stats_only_null.q    |    1 +
 .../queries/clientpositive/subquery_exists.q    |    1 +
 .../test/queries/clientpositive/subquery_in.q   |    1 +
 ql/src/test/queries/clientpositive/temp_table.q |    1 +
 .../clientpositive/tez_bmj_schema_evolution.q   |    1 +
 ql/src/test/queries/clientpositive/tez_dml.q    |    1 +
 .../clientpositive/tez_dynpart_hashjoin_1.q     |  101 +
 .../clientpositive/tez_dynpart_hashjoin_2.q     |   83 +
 ql/src/test/queries/clientpositive/tez_join.q   |    1 +
 .../test/queries/clientpositive/tez_join_hash.q |    1 +
 .../clientpositive/tez_join_result_complex.q    |  139 +
 .../queries/clientpositive/tez_join_tests.q     |    1 +
 .../queries/clientpositive/tez_joins_explain.q  |    1 +
 .../test/queries/clientpositive/tez_self_join.q |   33 +
 ql/src/test/queries/clientpositive/tez_smb_1.q  |   32 +
 .../test/queries/clientpositive/tez_smb_main.q  |    1 +
 ql/src/test/queries/clientpositive/tez_union.q  |    1 +
 ql/src/test/queries/clientpositive/tez_union2.q |    1 +
 .../tez_union_dynamic_partition.q               |   22 +
 .../queries/clientpositive/tez_union_group_by.q |    1 +
 .../clientpositive/tez_union_multiinsert.q      |    1 +
 .../tez_vector_dynpart_hashjoin_1.q             |  102 +
 .../tez_vector_dynpart_hashjoin_2.q             |   84 +
 ql/src/test/queries/clientpositive/transform1.q |    1 +
 .../clientpositive/udaf_percentile_approx_23.q  |    2 -
 ql/src/test/queries/clientpositive/union2.q     |    1 +
 ql/src/test/queries/clientpositive/union3.q     |    1 +
 ql/src/test/queries/clientpositive/union35.q    |    2 -
 ql/src/test/queries/clientpositive/union4.q     |    1 +
 ql/src/test/queries/clientpositive/union5.q     |    1 +
 ql/src/test/queries/clientpositive/union6.q     |    1 +
 ql/src/test/queries/clientpositive/union7.q     |    1 +
 ql/src/test/queries/clientpositive/union8.q     |    1 +
 ql/src/test/queries/clientpositive/union9.q     |    1 +
 .../queries/clientpositive/unionDistinct_1.q    |    1 +
 .../queries/clientpositive/update_all_types.q   |    2 +
 .../queries/clientpositive/update_orig_table.q  |    2 +
 .../queries/clientpositive/vector_aggregate_9.q |    1 +
 .../vector_aggregate_without_gby.q              |   14 +
 .../queries/clientpositive/vector_between_in.q  |    1 +
 .../clientpositive/vector_binary_join_groupby.q |    1 +
 .../test/queries/clientpositive/vector_bucket.q |    1 +
 .../clientpositive/vector_cast_constant.q       |    1 +
 .../test/queries/clientpositive/vector_char_2.q |    1 +
 .../test/queries/clientpositive/vector_char_4.q |    1 +
 .../clientpositive/vector_char_mapjoin1.q       |    1 +
 .../queries/clientpositive/vector_char_simple.q |    1 +
 .../queries/clientpositive/vector_coalesce.q    |   52 +-
 .../queries/clientpositive/vector_coalesce_2.q  |    1 +
 .../clientpositive/vector_count_distinct.q      |    1 +
 .../queries/clientpositive/vector_data_types.q  |    1 +
 .../test/queries/clientpositive/vector_date_1.q |    1 +
 .../queries/clientpositive/vector_decimal_1.q   |    1 +
 .../clientpositive/vector_decimal_10_0.q        |    1 +
 .../queries/clientpositive/vector_decimal_2.q   |    1 +
 .../clientpositive/vector_decimal_aggregate.q   |    1 +
 .../clientpositive/vector_decimal_cast.q        |    1 +
 .../clientpositive/vector_decimal_expressions.q |   12 +-
 .../clientpositive/vector_decimal_mapjoin.q     |    1 +
 .../clientpositive/vector_decimal_math_funcs.q  |    1 +
 .../clientpositive/vector_decimal_precision.q   |    1 +
 .../clientpositive/vector_decimal_round.q       |    1 +
 .../clientpositive/vector_decimal_round_2.q     |    1 +
 .../queries/clientpositive/vector_decimal_udf.q |    1 +
 .../clientpositive/vector_decimal_udf2.q        |    1 +
 .../queries/clientpositive/vector_distinct_2.q  |    1 +
 ql/src/test/queries/clientpositive/vector_elt.q |    1 +
 .../queries/clientpositive/vector_groupby_3.q   |    1 +
 .../clientpositive/vector_groupby_reduce.q      |    4 +
 .../clientpositive/vector_grouping_sets.q       |    1 +
 .../queries/clientpositive/vector_if_expr.q     |    1 +
 .../queries/clientpositive/vector_inner_join.q  |    1 +
 .../queries/clientpositive/vector_interval_1.q  |    1 +
 .../queries/clientpositive/vector_interval_2.q  |    1 +
 .../clientpositive/vector_interval_mapjoin.q    |    1 +
 .../test/queries/clientpositive/vector_join30.q |    1 +
 .../clientpositive/vector_left_outer_join.q     |    1 +
 .../clientpositive/vector_left_outer_join2.q    |    1 +
 .../clientpositive/vector_leftsemi_mapjoin.q    |    1 +
 .../clientpositive/vector_mapjoin_reduce.q      |    1 +
 .../vector_mr_diff_schema_alias.q               |    1 +
 .../clientpositive/vector_multi_insert.q        |    1 +
 .../vector_non_string_partition.q               |    1 +
 .../clientpositive/vector_null_projection.q     |    1 +
 .../clientpositive/vector_nullsafe_join.q       |    1 +
 .../queries/clientpositive/vector_orderby_5.q   |    1 +
 .../queries/clientpositive/vector_outer_join0.q |    1 +
 .../queries/clientpositive/vector_outer_join1.q |   20 +-
 .../queries/clientpositive/vector_outer_join2.q |   19 +-
 .../queries/clientpositive/vector_outer_join3.q |   18 +-
 .../queries/clientpositive/vector_outer_join4.q |   17 +-
 .../queries/clientpositive/vector_outer_join5.q |    1 +
 .../vector_partition_diff_num_cols.q            |    1 +
 .../vector_partitioned_date_time.q              |    5 +-
 .../vector_partitioned_date_time_win.q          |  129 +
 .../vector_reduce_groupby_decimal.q             |    1 +
 .../clientpositive/vector_string_concat.q       |    1 +
 .../queries/clientpositive/vector_varchar_4.q   |    1 +
 .../clientpositive/vector_varchar_mapjoin1.q    |    1 +
 .../clientpositive/vector_varchar_simple.q      |    1 +
 .../queries/clientpositive/vectorization_0.q    |    3 +
 .../queries/clientpositive/vectorization_1.q    |    3 +
 .../queries/clientpositive/vectorization_10.q   |    3 +
 .../queries/clientpositive/vectorization_11.q   |    3 +
 .../queries/clientpositive/vectorization_12.q   |    3 +
 .../queries/clientpositive/vectorization_13.q   |  142 +-
 .../queries/clientpositive/vectorization_14.q   |    3 +
 .../queries/clientpositive/vectorization_15.q   |    3 +
 .../queries/clientpositive/vectorization_16.q   |    1 +
 .../queries/clientpositive/vectorization_17.q   |   51 +
 .../queries/clientpositive/vectorization_2.q    |    3 +
 .../queries/clientpositive/vectorization_3.q    |    3 +
 .../queries/clientpositive/vectorization_4.q    |    3 +
 .../queries/clientpositive/vectorization_5.q    |    3 +
 .../queries/clientpositive/vectorization_6.q    |    3 +
 .../queries/clientpositive/vectorization_7.q    |   80 +-
 .../queries/clientpositive/vectorization_8.q    |   81 +-
 .../queries/clientpositive/vectorization_9.q    |   39 +-
 .../clientpositive/vectorization_decimal_date.q |    1 +
 .../queries/clientpositive/vectorization_div0.q |    1 +
 .../clientpositive/vectorization_limit.q        |    1 +
 .../clientpositive/vectorization_part_project.q |    1 +
 .../clientpositive/vectorization_part_varchar.q |    7 +
 .../clientpositive/vectorization_pushdown.q     |    1 +
 .../vectorization_short_regress.q               |  420 +-
 .../clientpositive/vectorized_bucketmapjoin1.q  |    1 +
 .../queries/clientpositive/vectorized_case.q    |    1 +
 .../queries/clientpositive/vectorized_casts.q   |    3 +
 .../queries/clientpositive/vectorized_context.q |    1 +
 .../clientpositive/vectorized_date_funcs.q      |    3 +
 .../clientpositive/vectorized_distinct_gby.q    |    1 +
 .../vectorized_dynamic_partition_pruning.q      |    1 +
 .../queries/clientpositive/vectorized_mapjoin.q |    1 +
 .../clientpositive/vectorized_math_funcs.q      |    1 +
 .../clientpositive/vectorized_nested_mapjoin.q  |    1 +
 .../queries/clientpositive/vectorized_parquet.q |    1 +
 .../clientpositive/vectorized_shufflejoin.q     |    1 +
 .../clientpositive/vectorized_string_funcs.q    |    1 +
 .../clientpositive/vectorized_timestamp_funcs.q |    1 +
 ql/src/test/queries/clientpositive/windowing.q  |    2 +-
 .../clientpositive/windowing_windowspec3.q      |   62 +
 .../resources/orc-file-dump-bloomfilter.out     |   92 +-
 .../resources/orc-file-dump-bloomfilter2.out    |   92 +-
 .../orc-file-dump-dictionary-threshold.out      |   76 +-
 ql/src/test/resources/orc-file-dump.json        |  108 +-
 ql/src/test/resources/orc-file-dump.out         |   84 +-
 ql/src/test/resources/orc-file-has-null.out     |   62 +-
 .../alter_numbuckets_partitioned_table.q.out    |    8 -
 .../results/beelinepositive/create_like.q.out   |    3 -
 .../results/beelinepositive/create_like2.q.out  |    1 -
 .../beelinepositive/create_like_view.q.out      |    4 -
 .../beelinepositive/create_skewed_table1.q.out  |    3 -
 .../results/beelinepositive/create_view.q.out   |   14 -
 .../create_view_partitioned.q.out               |    3 -
 ql/src/test/results/beelinepositive/ctas.q.out  |    5 -
 .../describe_formatted_view_partitioned.q.out   |    1 -
 .../beelinepositive/describe_table.q.out        |    3 -
 .../test/results/beelinepositive/merge3.q.out   |    1 -
 .../part_inherit_tbl_props.q.out                |    1 -
 .../part_inherit_tbl_props_empty.q.out          |    1 -
 .../part_inherit_tbl_props_with_star.q.out      |    1 -
 .../results/beelinepositive/protectmode2.q.out  |    2 -
 .../test/results/beelinepositive/stats1.q.out   |    2 -
 .../test/results/beelinepositive/stats10.q.out  |    3 -
 .../test/results/beelinepositive/stats11.q.out  |    4 -
 .../test/results/beelinepositive/stats12.q.out  |    5 -
 .../test/results/beelinepositive/stats13.q.out  |    6 -
 .../test/results/beelinepositive/stats14.q.out  |    5 -
 .../test/results/beelinepositive/stats15.q.out  |    5 -
 .../test/results/beelinepositive/stats16.q.out  |    2 -
 .../test/results/beelinepositive/stats18.q.out  |    2 -
 .../test/results/beelinepositive/stats2.q.out   |    2 -
 .../test/results/beelinepositive/stats3.q.out   |    2 -
 .../test/results/beelinepositive/stats4.q.out   |    6 -
 .../test/results/beelinepositive/stats5.q.out   |    1 -
 .../test/results/beelinepositive/stats6.q.out   |    5 -
 .../test/results/beelinepositive/stats7.q.out   |    3 -
 .../test/results/beelinepositive/stats8.q.out   |   10 -
 .../test/results/beelinepositive/stats9.q.out   |    1 -
 .../beelinepositive/stats_empty_partition.q.out |    1 -
 .../clientnegative/alter_file_format.q.out      |    1 -
 .../alter_view_as_select_with_partition.q.out   |    1 -
 .../clientnegative/exchange_partition.q.out     |   54 +
 .../clientnegative/load_orc_negative1.q.out     |    9 +
 .../clientnegative/load_orc_negative2.q.out     |   25 +
 .../clientnegative/load_orc_negative_part.q.out |   52 +
 .../lockneg_try_lock_db_in_use.q.out            |    9 +-
 .../ptf_negative_NoWindowDefn.q.out             |    1 +
 .../stats_partialscan_autogether.q.out          |    2 -
 .../clientnegative/udf_assert_true.q.out        |   12 +-
 .../clientnegative/udf_assert_true2.q.out       |    6 +-
 .../clientpositive/alter_file_format.q.out      |   19 -
 .../clientpositive/alter_merge_orc.q.out        |   24 +-
 .../clientpositive/alter_merge_stats_orc.q.out  |   30 +-
 .../alter_numbuckets_partitioned_table.q.out    |   16 -
 .../alter_numbuckets_partitioned_table2.q.out   |   27 -
 ...lter_numbuckets_partitioned_table2_h23.q.out |   27 -
 ...alter_numbuckets_partitioned_table_h23.q.out |   16 -
 .../alter_partition_change_col.q.out            |    8 +-
 .../alter_partition_clusterby_sortby.q.out      |    7 -
 .../clientpositive/alter_skewed_table.q.out     |    6 -
 .../clientpositive/alter_table_cascade.q.out    |    8 +-
 .../clientpositive/alter_table_not_sorted.q.out |    2 -
 .../clientpositive/alter_table_serde2.q.out     |    6 -
 .../clientpositive/alter_view_as_select.q.out   |    3 -
 .../clientpositive/annotate_stats_groupby.q.out |   28 +-
 .../annotate_stats_groupby2.q.out               |    8 +-
 .../annotate_stats_join_pkfk.q.out              |   20 +-
 .../clientpositive/annotate_stats_part.q.out    |    6 +-
 .../clientpositive/annotate_stats_select.q.out  |   52 +-
 .../clientpositive/annotate_stats_table.q.out   |    4 +-
 .../clientpositive/authorization_index.q.out    |    1 -
 .../results/clientpositive/auto_join12.q.out    |   54 +-
 .../results/clientpositive/auto_join13.q.out    |   26 +-
 .../results/clientpositive/auto_join14.q.out    |   12 +-
 .../results/clientpositive/auto_join5.q.out     |    8 +-
 .../results/clientpositive/auto_join8.q.out     |    2 +-
 .../auto_join_without_localtask.q.out           |  218 +-
 .../clientpositive/autogen_colalias.q.out       |    4 +-
 .../clientpositive/avro_timestamp.q.java1.7.out |    8 +-
 .../clientpositive/avro_timestamp.q.java1.8.out |    8 +-
 .../avro_timestamp_win.q.java1.7.out            |  134 +
 .../avro_timestamp_win.q.java1.8.out            |  134 +
 .../results/clientpositive/ba_table_union.q.out |   16 +-
 .../test/results/clientpositive/bucket5.q.out   |    1 -
 .../bucketsortoptimize_insert_7.q.out           |    2 +-
 .../clientpositive/cbo_rp_auto_join0.q.out      |   32 +-
 .../clientpositive/cbo_rp_auto_join1.q.out      |  619 +-
 .../results/clientpositive/cbo_rp_join0.q.out   | 6886 ++++++++++++++++++
 .../results/clientpositive/cbo_rp_join1.q.out   |  426 ++
 .../results/clientpositive/cbo_subq_in.q.out    |    2 +
 .../results/clientpositive/cbo_windowing.q.out  |   88 +-
 .../test/results/clientpositive/cluster.q.out   |   20 +-
 .../test/results/clientpositive/combine2.q.out  |   16 +-
 .../constantPropagateForSubQuery.q.out          |   38 +-
 .../clientpositive/correlationoptimizer15.q.out |  120 +-
 .../clientpositive/correlationoptimizer6.q.out  | 1011 ++-
 ql/src/test/results/clientpositive/cp_sel.q.out |  195 +
 .../create_alter_list_bucketing_table1.q.out    |    7 -
 .../results/clientpositive/create_like.q.out    |    9 -
 .../results/clientpositive/create_like2.q.out   |    1 -
 .../clientpositive/create_like_tbl_props.q.out  |    5 -
 .../clientpositive/create_like_view.q.out       |    4 -
 .../clientpositive/create_or_replace_view.q.out |    5 -
 .../clientpositive/create_skewed_table1.q.out   |    3 -
 .../results/clientpositive/create_view.q.out    |   14 -
 .../create_view_partitioned.q.out               |    3 -
 .../clientpositive/create_view_translate.q.out  |    2 -
 .../results/clientpositive/cross_join.q.out     |    8 +-
 ql/src/test/results/clientpositive/ctas.q.out   |    5 -
 .../results/clientpositive/ctas_colname.q.out   |    7 -
 .../results/clientpositive/ctas_hadoop20.q.out  |    5 -
 .../ctas_uses_database_location.q.out           |    1 -
 .../clientpositive/database_location.q.out      |    2 -
 .../results/clientpositive/decimal_serde.q.out  |    2 -
 .../clientpositive/default_file_format.q.out    |    5 -
 .../describe_comment_indent.q.out               |    1 -
 .../describe_comment_nonascii.q.out             |    1 -
 .../describe_formatted_view_partitioned.q.out   |    2 -
 .../clientpositive/describe_syntax.q.out        |    6 -
 .../results/clientpositive/describe_table.q.out |    7 -
 .../dynpart_sort_opt_bucketing.q.out            |  277 +
 .../dynpart_sort_opt_vectorization.q.out        |   48 +-
 .../dynpart_sort_optimization.q.out             |   32 -
 .../dynpart_sort_optimization2.q.out            |   32 +-
 .../dynpart_sort_optimization_acid.q.out        |   60 +-
 .../clientpositive/encoding_nonutf8.q.out       |   36 +
 .../encrypted/encryption_drop_partition.q.out   |   81 +
 .../encrypted/encryption_drop_table.q.out       |    2 +-
 .../encryption_insert_partition_dynamic.q.out   |   18 +-
 .../encryption_insert_partition_static.q.out    |  739 +-
 .../encrypted/encryption_insert_values.q.out    |    1 -
 .../clientpositive/exim_hidden_files.q.out      |    1 -
 .../extrapolate_part_stats_full.q.out           |   24 +-
 .../extrapolate_part_stats_partial.q.out        |   76 +-
 .../extrapolate_part_stats_partial_ndv.q.out    |   38 +-
 .../clientpositive/filter_join_breaktask.q.out  |   12 +-
 .../clientpositive/fold_eq_with_case_when.q.out |    6 +-
 .../test/results/clientpositive/fold_when.q.out |    2 +-
 .../clientpositive/fouter_join_ppr.q.out        | 1694 +++++
 .../test/results/clientpositive/groupby10.q.out |  140 +-
 .../test/results/clientpositive/groupby11.q.out |   70 +-
 .../test/results/clientpositive/groupby2.q.out  |   33 +-
 .../clientpositive/groupby2_map_skew.q.out      |   33 +-
 .../test/results/clientpositive/groupby8.q.out  |  140 +-
 .../clientpositive/groupby8_map_skew.q.out      |   70 +-
 ...pby_complex_types_multi_single_reducer.q.out |   68 +-
 .../results/clientpositive/groupby_cube1.q.out  |   29 +-
 .../clientpositive/groupby_grouping_sets6.q.out |   46 +-
 .../clientpositive/groupby_rollup1.q.out        |   29 +-
 .../clientpositive/groupby_sort_1_23.q.out      |   10 +-
 .../clientpositive/groupby_sort_skew_1_23.q.out |   10 +-
 ql/src/test/results/clientpositive/having.q.out |   28 +-
 .../clientpositive/index_auto_mult_tables.q.out |   12 +
 .../index_auto_mult_tables_compact.q.out        |    9 +
 .../clientpositive/index_auto_partitioned.q.out |    9 +
 .../clientpositive/index_auto_unused.q.out      |    4 +-
 .../clientpositive/index_auto_update.q.out      |    2 +
 .../results/clientpositive/index_bitmap.q.out   |   24 +
 .../index_bitmap_auto_partitioned.q.out         |   12 +
 .../clientpositive/index_bitmap_rc.q.out        |   24 +
 .../results/clientpositive/index_compact.q.out  |   18 +
 .../clientpositive/index_compact_2.q.out        |   18 +
 .../clientpositive/index_skewtable.q.out        |    1 -
 .../clientpositive/infer_bucket_sort.q.out      |   50 -
 .../infer_bucket_sort_bucketed_table.q.out      |    2 -
 .../infer_bucket_sort_convert_join.q.out        |    4 -
 .../infer_bucket_sort_dyn_part.q.out            |   16 -
 .../infer_bucket_sort_grouping_operators.q.out  |   12 -
 .../infer_bucket_sort_list_bucket.q.out         |    4 -
 .../infer_bucket_sort_map_operators.q.out       |    8 -
 .../infer_bucket_sort_merge.q.out               |    4 -
 .../infer_bucket_sort_multi_insert.q.out        |   16 -
 .../infer_bucket_sort_num_buckets.q.out         |    4 -
 .../infer_bucket_sort_reducers_power_two.q.out  |   12 -
 ql/src/test/results/clientpositive/input7.q.out |    2 +-
 .../results/clientpositive/input_part10.q.out   |    5 +-
 .../results/clientpositive/insert_into5.q.out   |    5 +-
 .../insert_non_utf8_encoding_table.q.out        |   89 +
 ql/src/test/results/clientpositive/join12.q.out |   28 +-
 ql/src/test/results/clientpositive/join13.q.out |   32 +-
 ql/src/test/results/clientpositive/join14.q.out |   14 +-
 ql/src/test/results/clientpositive/join28.q.out |    2 +-
 ql/src/test/results/clientpositive/join32.q.out |   38 +-
 .../clientpositive/join32_lessSize.q.out        |  128 +-
 ql/src/test/results/clientpositive/join33.q.out |   38 +-
 ql/src/test/results/clientpositive/join34.q.out |    2 +-
 ql/src/test/results/clientpositive/join35.q.out |    2 +-
 ql/src/test/results/clientpositive/join42.q.out |  312 +
 ql/src/test/results/clientpositive/join43.q.out |  648 ++
 ql/src/test/results/clientpositive/join5.q.out  |   20 +-
 ql/src/test/results/clientpositive/join8.q.out  |    2 +-
 .../clientpositive/join_alt_syntax.q.out        |  104 +-
 .../clientpositive/join_cond_pushdown_1.q.out   |   42 +-
 .../clientpositive/join_cond_pushdown_2.q.out   |   62 +-
 .../clientpositive/join_cond_pushdown_3.q.out   |   42 +-
 .../clientpositive/join_cond_pushdown_4.q.out   |   62 +-
 .../results/clientpositive/join_merging.q.out   |  117 +-
 .../results/clientpositive/join_nulls.q.out     |    2 +-
 .../results/clientpositive/lateral_view.q.out   |   68 +-
 .../clientpositive/lateral_view_explode2.q.out  |    6 +-
 .../clientpositive/lateral_view_noalias.q.out   |   22 +-
 .../results/clientpositive/lb_fs_stats.q.out    |    2 -
 .../results/clientpositive/limit_pushdown.q.out |   98 +-
 .../test/results/clientpositive/lineage1.q.out  |    4 +-
 .../test/results/clientpositive/lineage2.q.out  |  677 ++
 .../test/results/clientpositive/lineage3.q.out  |  293 +
 .../clientpositive/list_bucket_dml_1.q.out      |    4 -
 .../list_bucket_dml_10.q.java1.7.out            |    2 -
 .../list_bucket_dml_10.q.java1.8.out            |    2 -
 .../list_bucket_dml_11.q.java1.7.out            |    2 -
 .../list_bucket_dml_11.q.java1.8.out            |    2 -
 .../list_bucket_dml_12.q.java1.7.out            |    4 +-
 .../list_bucket_dml_12.q.java1.8.out            |    2 -
 .../list_bucket_dml_13.q.java1.7.out            |    4 +-
 .../list_bucket_dml_13.q.java1.8.out            |    2 -
 .../clientpositive/list_bucket_dml_14.q.out     |    1 -
 .../list_bucket_dml_2.q.java1.7.out             |    2 -
 .../list_bucket_dml_2.q.java1.8.out             |    2 -
 .../clientpositive/list_bucket_dml_3.q.out      |    2 -
 .../list_bucket_dml_4.q.java1.7.out             |    4 -
 .../list_bucket_dml_4.q.java1.8.out             |    4 -
 .../list_bucket_dml_5.q.java1.7.out             |    4 -
 .../list_bucket_dml_5.q.java1.8.out             |    4 -
 .../list_bucket_dml_6.q.java1.7.out             |    8 -
 .../list_bucket_dml_6.q.java1.8.out             |    8 -
 .../clientpositive/list_bucket_dml_7.q.out      |    8 -
 .../list_bucket_dml_8.q.java1.7.out             |    6 -
 .../list_bucket_dml_8.q.java1.8.out             |    6 -
 .../list_bucket_dml_9.q.java1.7.out             |    4 -
 .../list_bucket_dml_9.q.java1.8.out             |    4 -
 .../list_bucket_query_multiskew_1.q.out         |    2 -
 .../list_bucket_query_multiskew_2.q.out         |    2 -
 .../list_bucket_query_multiskew_3.q.out         |    6 -
 .../list_bucket_query_oneskew_1.q.out           |    2 -
 .../list_bucket_query_oneskew_2.q.out           |    2 -
 .../list_bucket_query_oneskew_3.q.out           |    2 -
 .../clientpositive/load_dyn_part13.q.out        |    8 +-
 .../clientpositive/load_dyn_part14.q.out        |   63 +-
 .../test/results/clientpositive/load_orc.q.out  |   43 +
 .../results/clientpositive/load_orc_part.q.out  |   70 +
 .../clientpositive/louter_join_ppr.q.out        |   74 +-
 .../clientpositive/mapjoin_mapjoin.q.out        |  120 +-
 .../clientpositive/mapjoin_subquery.q.out       |    4 +-
 ql/src/test/results/clientpositive/merge3.q.out |    1 -
 .../results/clientpositive/multiMapJoin1.q.out  |   10 +-
 .../results/clientpositive/multi_insert.q.out   |   32 +-
 ...i_insert_move_tasks_share_dependencies.q.out |   32 +-
 .../results/clientpositive/optional_outer.q.out |   36 +-
 .../results/clientpositive/orc_analyze.q.out    |   48 -
 .../results/clientpositive/orc_create.q.out     |    6 -
 .../orc_dictionary_threshold.q.out              |    2 +-
 .../results/clientpositive/orc_merge9.q.out     |  186 +
 .../clientpositive/outer_join_ppr.q.java1.7.out |  168 +-
 .../clientpositive/parallel_orderby.q.out       |    2 -
 .../parquet_array_null_element.q.out            |    1 -
 .../results/clientpositive/parquet_create.q.out |    1 -
 .../clientpositive/parquet_partitioned.q.out    |    1 -
 .../results/clientpositive/parquet_serde.q.out  |    5 -
 .../clientpositive/part_inherit_tbl_props.q.out |    2 -
 .../part_inherit_tbl_props_empty.q.out          |    2 -
 .../part_inherit_tbl_props_with_star.q.out      |    2 -
 .../partition_coltype_literals.q.out            |   16 -
 .../clientpositive/partition_timestamp.q.out    |    8 +-
 .../clientpositive/partition_timestamp2.q.out   |    8 +-
 .../partition_timestamp2_win.q.out              |  399 +
 .../partition_timestamp_win.q.out               |  316 +
 .../results/clientpositive/ppd_gby_join.q.out   |  104 +-
 .../test/results/clientpositive/ppd_join.q.out  |  106 +-
 .../test/results/clientpositive/ppd_join2.q.out |   88 +-
 .../test/results/clientpositive/ppd_join3.q.out |  114 +-
 .../clientpositive/ppd_outer_join4.q.out        |   88 +-
 .../results/clientpositive/ppd_random.q.out     |   80 +-
 .../results/clientpositive/ppd_udf_case.q.out   |   40 +-
 .../results/clientpositive/ppd_union_view.q.out |   78 +-
 .../results/clientpositive/protectmode2.q.out   |    2 -
 ql/src/test/results/clientpositive/ptf.q.out    |   28 +-
 .../clientpositive/rcfile_default_format.q.out  |    8 -
 .../clientpositive/rcfile_null_value.q.out      |   20 +-
 .../clientpositive/router_join_ppr.q.out        |  170 +-
 .../clientpositive/selectDistinctStar.q.out     |    2 -
 .../clientpositive/select_same_col.q.out        |    8 +-
 .../test/results/clientpositive/skewjoin.q.out  |   46 +-
 .../clientpositive/smb_join_partition_key.q.out |  128 +
 .../clientpositive/spark/alter_merge_orc.q.out  |   24 +-
 .../spark/alter_merge_stats_orc.q.out           |   30 +-
 .../spark/annotate_stats_join.q.out             |  204 +-
 .../clientpositive/spark/auto_join1.q.out       |   60 +-
 .../clientpositive/spark/auto_join10.q.out      |   28 +-
 .../clientpositive/spark/auto_join11.q.out      |   32 +-
 .../clientpositive/spark/auto_join12.q.out      |   54 +-
 .../clientpositive/spark/auto_join13.q.out      |   40 +-
 .../clientpositive/spark/auto_join14.q.out      |   70 +-
 .../clientpositive/spark/auto_join16.q.out      |   68 +-
 .../clientpositive/spark/auto_join17.q.out      |   60 +-
 .../clientpositive/spark/auto_join18.q.out      |   30 +-
 .../spark/auto_join18_multi_distinct.q.out      |   30 +-
 .../clientpositive/spark/auto_join19.q.out      |   56 +-
 .../clientpositive/spark/auto_join26.q.out      |   68 +-
 .../clientpositive/spark/auto_join27.q.out      |   24 +-
 .../clientpositive/spark/auto_join4.q.out       |   14 +-
 .../clientpositive/spark/auto_join5.q.out       |   14 +-
 .../clientpositive/spark/auto_join6.q.out       |   10 +-
 .../clientpositive/spark/auto_join8.q.out       |   14 +-
 .../clientpositive/spark/auto_join9.q.out       |   56 +-
 .../spark/auto_join_filters.q.out               |    4 +-
 .../clientpositive/spark/auto_join_nulls.q.out  |    6 +-
 .../spark/auto_join_without_localtask.q.out     |  122 +-
 .../results/clientpositive/spark/bucket5.q.out  |    1 -
 .../spark/bucketsortoptimize_insert_7.q.out     |   18 +-
 .../clientpositive/spark/cbo_subq_in.q.out      |    2 +
 .../spark/constprog_partitioner.q.out           |   80 +-
 .../results/clientpositive/spark/count.q.out    |   14 +-
 .../clientpositive/spark/cross_join.q.out       |   74 +-
 .../spark/cross_product_check_1.q.out           |   40 +-
 .../spark/cross_product_check_2.q.out           |   38 +-
 .../results/clientpositive/spark/ctas.q.out     |    5 -
 .../spark/filter_join_breaktask.q.out           |   12 +-
 .../results/clientpositive/spark/groupby1.q.out |    8 +-
 .../clientpositive/spark/groupby10.q.out        |   88 +-
 .../clientpositive/spark/groupby11.q.out        |   44 +-
 .../results/clientpositive/spark/groupby2.q.out |   17 +-
 .../results/clientpositive/spark/groupby4.q.out |    6 +-
 .../results/clientpositive/spark/groupby8.q.out |   88 +-
 .../spark/groupby8_map_skew.q.out               |   44 +-
 ...pby_complex_types_multi_single_reducer.q.out |   90 +-
 .../clientpositive/spark/groupby_cube1.q.out    |   41 +-
 .../clientpositive/spark/groupby_position.q.out |  130 +-
 .../clientpositive/spark/groupby_rollup1.q.out  |   37 +-
 .../spark/groupby_sort_1_23.q.out               |   92 +-
 .../spark/groupby_sort_skew_1_23.q.out          |   92 +-
 .../results/clientpositive/spark/having.q.out   |   94 +-
 .../spark/index_auto_self_join.q.out            |  116 +-
 .../infer_bucket_sort_bucketed_table.q.out      |    2 -
 .../spark/infer_bucket_sort_convert_join.q.out  |    4 -
 .../spark/infer_bucket_sort_map_operators.q.out |   13 +-
 .../spark/infer_bucket_sort_merge.q.out         |    8 +-
 .../spark/infer_bucket_sort_num_buckets.q.out   |    4 -
 .../infer_bucket_sort_reducers_power_two.q.out  |   12 -
 .../clientpositive/spark/innerjoin.q.out        |   54 +-
 .../results/clientpositive/spark/join1.q.out    |   38 +-
 .../results/clientpositive/spark/join10.q.out   |   28 +-
 .../results/clientpositive/spark/join11.q.out   |   18 +-
 .../results/clientpositive/spark/join12.q.out   |   26 +-
 .../results/clientpositive/spark/join13.q.out   |   82 +-
 .../results/clientpositive/spark/join14.q.out   |   60 +-
 .../results/clientpositive/spark/join16.q.out   |   36 +-
 .../results/clientpositive/spark/join17.q.out   |   52 +-
 .../results/clientpositive/spark/join18.q.out   |   12 +-
 .../spark/join18_multi_distinct.q.out           |   12 +-
 .../results/clientpositive/spark/join28.q.out   |    2 +-
 .../results/clientpositive/spark/join29.q.out   |   66 +-
 .../results/clientpositive/spark/join31.q.out   |  104 +-
 .../results/clientpositive/spark/join32.q.out   |  104 +-
 .../clientpositive/spark/join32_lessSize.q.out  |  242 +-
 .../results/clientpositive/spark/join33.q.out   |  104 +-
 .../results/clientpositive/spark/join34.q.out   |   36 +-
 .../results/clientpositive/spark/join35.q.out   |   92 +-
 .../results/clientpositive/spark/join4.q.out    |   16 +-
 .../results/clientpositive/spark/join40.q.out   |  116 +-
 .../results/clientpositive/spark/join5.q.out    |   16 +-
 .../results/clientpositive/spark/join6.q.out    |   10 +-
 .../results/clientpositive/spark/join8.q.out    |   16 +-
 .../results/clientpositive/spark/join9.q.out    |   42 +-
 .../clientpositive/spark/join_alt_syntax.q.out  |  314 +-
 .../spark/join_cond_pushdown_1.q.out            |   70 +-
 .../spark/join_cond_pushdown_2.q.out            |  134 +-
 .../spark/join_cond_pushdown_3.q.out            |   70 +-
 .../spark/join_cond_pushdown_4.q.out            |  134 +-
 .../clientpositive/spark/join_merging.q.out     |  115 +-
 .../results/clientpositive/spark/join_vc.q.out  |   40 +-
 .../spark/lateral_view_explode2.q.out           |    6 +-
 .../spark/limit_partition_metadataonly.q.out    |    4 +-
 .../clientpositive/spark/limit_pushdown.q.out   |   58 +-
 .../spark/list_bucket_dml_10.q.java1.7.out      |    2 -
 .../spark/list_bucket_dml_10.q.java1.8.out      |    2 -
 .../spark/list_bucket_dml_2.q.java1.7.out       |    2 -
 .../spark/list_bucket_dml_2.q.java1.8.out       |    2 -
 .../spark/list_bucket_dml_2.q.out               |  Bin 28747 -> 28667 bytes
 .../clientpositive/spark/load_dyn_part13.q.out  |   12 +-
 .../clientpositive/spark/load_dyn_part14.q.out  |   57 +-
 .../clientpositive/spark/louter_join_ppr.q.out  |  454 +-
 .../clientpositive/spark/mapjoin_mapjoin.q.out  |  202 +-
 .../clientpositive/spark/mapjoin_subquery.q.out |    4 +-
 .../results/clientpositive/spark/merge1.q.out   |    4 +-
 .../results/clientpositive/spark/merge2.q.out   |    4 +-
 .../spark/metadata_only_queries.q.out           |   12 +-
 .../clientpositive/spark/multi_insert.q.out     |   32 +-
 ...i_insert_move_tasks_share_dependencies.q.out |   32 +-
 .../clientpositive/spark/orc_analyze.q.out      |   22 -
 .../results/clientpositive/spark/order2.q.out   |    2 +-
 .../spark/outer_join_ppr.q.java1.7.out          |  292 +-
 .../spark/outer_join_ppr.q.java1.8.out          |  366 +-
 .../clientpositive/spark/parallel_join1.q.out   |   38 +-
 .../clientpositive/spark/parallel_orderby.q.out |    2 -
 .../test/results/clientpositive/spark/pcr.q.out |    4 +-
 .../clientpositive/spark/ppd_gby_join.q.out     |   90 +-
 .../results/clientpositive/spark/ppd_join.q.out |   96 +-
 .../clientpositive/spark/ppd_join2.q.out        |   86 +-
 .../clientpositive/spark/ppd_join3.q.out        |  116 +-
 .../clientpositive/spark/ppd_join_filter.q.out  |  442 +-
 .../clientpositive/spark/ppd_outer_join1.q.out  |  120 +-
 .../clientpositive/spark/ppd_outer_join2.q.out  |  158 +-
 .../clientpositive/spark/ppd_outer_join3.q.out  |  154 +-
 .../clientpositive/spark/ppd_outer_join4.q.out  |  124 +-
 .../test/results/clientpositive/spark/ptf.q.out |   28 +-
 .../spark/ql_rewrite_gbtoidx_cbo_1.q.out        |  208 +-
 .../clientpositive/spark/router_join_ppr.q.out  |  442 +-
 .../results/clientpositive/spark/semijoin.q.out |   26 +-
 .../results/clientpositive/spark/skewjoin.q.out |  206 +-
 .../clientpositive/spark/skewjoin_noskew.q.out  |   32 +-
 .../results/clientpositive/spark/stats1.q.out   |    4 +-
 .../results/clientpositive/spark/stats10.q.out  |    5 -
 .../results/clientpositive/spark/stats12.q.out  |    9 -
 .../results/clientpositive/spark/stats13.q.out  |   10 -
 .../results/clientpositive/spark/stats14.q.out  |    7 -
 .../results/clientpositive/spark/stats15.q.out  |    7 -
 .../results/clientpositive/spark/stats16.q.out  |    2 -
 .../results/clientpositive/spark/stats18.q.out  |    4 -
 .../results/clientpositive/spark/stats2.q.out   |    2 -
 .../results/clientpositive/spark/stats20.q.out  |    2 -
 .../results/clientpositive/spark/stats3.q.out   |    2 -
 .../results/clientpositive/spark/stats5.q.out   |    1 -
 .../results/clientpositive/spark/stats6.q.out   |    9 -
 .../results/clientpositive/spark/stats7.q.out   |    5 -
 .../results/clientpositive/spark/stats8.q.out   |   18 -
 .../results/clientpositive/spark/stats9.q.out   |    1 -
 .../clientpositive/spark/stats_counter.q.out    |    2 -
 .../spark/stats_counter_partitioned.q.out       |   16 -
 .../clientpositive/spark/stats_noscan_1.q.out   |   17 -
 .../clientpositive/spark/stats_noscan_2.q.out   |    6 -
 .../clientpositive/spark/stats_only_null.q.out  |   12 +-
 .../spark/stats_partscan_1_23.q.out             |    6 -
 .../results/clientpositive/spark/statsfs.q.out  |   14 -
 .../clientpositive/spark/subquery_exists.q.out  |   20 +-
 .../clientpositive/spark/subquery_in.q.out      |  230 +-
 .../spark/subquery_multiinsert.q.java1.8.out    |   14 +-
 .../spark/table_access_keys_stats.q.out         |   18 +-
 .../clientpositive/spark/temp_table.q.out       |    4 +-
 .../results/clientpositive/spark/union.q.out    |    4 +-
 .../results/clientpositive/spark/union10.q.out  |    6 +-
 .../results/clientpositive/spark/union11.q.out  |   82 +-
 .../results/clientpositive/spark/union12.q.out  |    2 +-
 .../results/clientpositive/spark/union13.q.out  |    2 +-
 .../results/clientpositive/spark/union14.q.out  |   52 +-
 .../results/clientpositive/spark/union15.q.out  |   80 +-
 .../results/clientpositive/spark/union2.q.out   |    2 +-
 .../results/clientpositive/spark/union20.q.out  |   10 +-
 .../results/clientpositive/spark/union22.q.out  |    4 +-
 .../results/clientpositive/spark/union24.q.out  |   36 +-
 .../results/clientpositive/spark/union25.q.out  |   38 +-
 .../results/clientpositive/spark/union28.q.out  |   82 +-
 .../results/clientpositive/spark/union29.q.out  |    4 +-
 .../results/clientpositive/spark/union30.q.out  |   82 +-
 .../results/clientpositive/spark/union33.q.out  |   16 +-
 .../results/clientpositive/spark/union4.q.out   |    4 +-
 .../results/clientpositive/spark/union5.q.out   |   54 +-
 .../results/clientpositive/spark/union6.q.out   |    2 +-
 .../results/clientpositive/spark/union7.q.out   |   52 +-
 .../results/clientpositive/spark/union8.q.out   |    4 +-
 .../results/clientpositive/spark/union9.q.out   |    4 +-
 .../clientpositive/spark/union_date_trim.q.out  |    4 +-
 .../clientpositive/spark/union_remove_1.q.out   |   13 +-
 .../clientpositive/spark/union_remove_10.q.out  |   11 +-
 .../clientpositive/spark/union_remove_11.q.out  |    3 +-
 .../clientpositive/spark/union_remove_12.q.out  |    3 +-
 .../clientpositive/spark/union_remove_13.q.out  |    1 -
 .../clientpositive/spark/union_remove_14.q.out  |    3 +-
 .../clientpositive/spark/union_remove_15.q.out  |   17 +-
 .../clientpositive/spark/union_remove_16.q.out  |   17 +-
 .../clientpositive/spark/union_remove_17.q.out  |    5 +-
 .../clientpositive/spark/union_remove_18.q.out  |   33 +-
 .../clientpositive/spark/union_remove_19.q.out  |  205 +-
 .../clientpositive/spark/union_remove_2.q.out   |   11 +-
 .../clientpositive/spark/union_remove_20.q.out  |   13 +-
 .../clientpositive/spark/union_remove_21.q.out  |   73 +-
 .../clientpositive/spark/union_remove_22.q.out  |   21 +-
 .../clientpositive/spark/union_remove_23.q.out  |    3 +-
 .../clientpositive/spark/union_remove_24.q.out  |   13 +-
 .../clientpositive/spark/union_remove_25.q.out  |   30 +-
 .../clientpositive/spark/union_remove_3.q.out   |    3 +-
 .../clientpositive/spark/union_remove_4.q.out   |   13 +-
 .../clientpositive/spark/union_remove_5.q.out   |   11 +-
 .../clientpositive/spark/union_remove_6.q.out   |    8 +-
 .../spark/union_remove_6_subq.q.out             |   16 +-
 .../clientpositive/spark/union_remove_7.q.out   |   13 +-
 .../clientpositive/spark/union_remove_8.q.out   |   11 +-
 .../clientpositive/spark/union_remove_9.q.out   |    7 +-
 .../clientpositive/spark/union_top_level.q.out  |  382 +-
 .../spark/vector_cast_constant.q.java1.7.out    |    6 +-
 .../spark/vector_cast_constant.q.java1.8.out    |    6 +-
 .../spark/vector_decimal_aggregate.q.out        |   12 +-
 .../spark/vector_distinct_2.q.out               |    4 +-
 .../clientpositive/spark/vector_elt.q.out       |    2 +-
 .../clientpositive/spark/vector_groupby_3.q.out |    6 +-
 .../spark/vector_mapjoin_reduce.q.out           |  261 +-
 .../clientpositive/spark/vector_orderby_5.q.out |    6 +-
 .../spark/vector_string_concat.q.out            |    6 +-
 .../clientpositive/spark/vectorization_0.q.out  |   28 +-
 .../clientpositive/spark/vectorization_1.q.out  |    8 +-
 .../clientpositive/spark/vectorization_10.q.out |  428 +-
 .../clientpositive/spark/vectorization_11.q.out |   74 +-
 .../clientpositive/spark/vectorization_12.q.out |  994 +--
 .../clientpositive/spark/vectorization_13.q.out |  430 +-
 .../clientpositive/spark/vectorization_14.q.out | 1182 +--
 .../clientpositive/spark/vectorization_15.q.out |   68 +-
 .../clientpositive/spark/vectorization_16.q.out |    8 +-
 .../clientpositive/spark/vectorization_17.q.out |  507 ++
 .../clientpositive/spark/vectorization_2.q.out  |    8 +-
 .../clientpositive/spark/vectorization_3.q.out  |    8 +-
 .../clientpositive/spark/vectorization_4.q.out  |    8 +-
 .../clientpositive/spark/vectorization_5.q.out  |    8 +-
 .../clientpositive/spark/vectorization_6.q.out  | 3042 ++++----
 .../clientpositive/spark/vectorization_7.q.out  |  328 +-
 .../clientpositive/spark/vectorization_8.q.out  |  314 +-
 .../clientpositive/spark/vectorization_9.q.out  |  965 ++-
 .../spark/vectorization_div0.q.out              |    6 +-
 .../spark/vectorization_part_project.q.out      |    2 +-
 .../spark/vectorization_pushdown.q.out          |    8 +-
 .../spark/vectorization_short_regress.q.out     | 1330 ++--
 .../clientpositive/spark/vectorized_case.q.out  |    2 +-
 .../spark/vectorized_mapjoin.q.out              |   60 +-
 .../clientpositive/spark/vectorized_ptf.q.out   |  136 +-
 .../spark/vectorized_shufflejoin.q.out          |   52 +-
 .../spark/vectorized_timestamp_funcs.q.out      |   12 +-
 .../clientpositive/spark/windowing.q.out        |   44 +-
 ql/src/test/results/clientpositive/stats1.q.out |    2 -
 .../test/results/clientpositive/stats10.q.out   |    5 -
 .../test/results/clientpositive/stats11.q.out   |    8 -
 .../test/results/clientpositive/stats12.q.out   |    9 -
 .../test/results/clientpositive/stats13.q.out   |   10 -
 .../test/results/clientpositive/stats14.q.out   |    7 -
 .../test/results/clientpositive/stats15.q.out   |    7 -
 .../test/results/clientpositive/stats16.q.out   |    2 -
 .../test/results/clientpositive/stats18.q.out   |    4 -
 .../test/results/clientpositive/stats19.q.out   |   12 -
 ql/src/test/results/clientpositive/stats2.q.out |    2 -
 .../test/results/clientpositive/stats20.q.out   |    2 -
 ql/src/test/results/clientpositive/stats3.q.out |    2 -
 ql/src/test/results/clientpositive/stats4.q.out |   10 -
 ql/src/test/results/clientpositive/stats5.q.out |    1 -
 ql/src/test/results/clientpositive/stats6.q.out |    9 -
 ql/src/test/results/clientpositive/stats7.q.out |    5 -
 ql/src/test/results/clientpositive/stats8.q.out |   18 -
 ql/src/test/results/clientpositive/stats9.q.out |    1 -
 .../results/clientpositive/stats_counter.q.out  |    2 -
 .../stats_counter_partitioned.q.out             |   16 -
 .../clientpositive/stats_empty_partition.q.out  |    2 -
 .../clientpositive/stats_invalidation.q.out     |    2 -
 .../stats_list_bucket.q.java1.7.out             |    3 -
 .../stats_list_bucket.q.java1.8.out             |    3 -
 .../results/clientpositive/stats_noscan_1.q.out |   17 -
 .../results/clientpositive/stats_noscan_2.q.out |    6 -
 .../clientpositive/stats_only_null.q.out        |    4 -
 .../clientpositive/stats_partscan_1.q.out       |    6 -
 .../clientpositive/stats_partscan_1_23.q.out    |    6 -
 .../test/results/clientpositive/statsfs.q.out   |   14 -
 .../clientpositive/subquery_in_having.q.out     |  128 +-
 .../subquery_notin_having.q.java1.7.out         |   34 +-
 .../subquery_unqualcolumnrefs.q.out             |   32 +-
 .../temp_table_windowing_expressions.q.out      |    8 +-
 .../clientpositive/tez/alter_merge_orc.q.out    |   24 +-
 .../tez/alter_merge_stats_orc.q.out             |   30 +-
 .../clientpositive/tez/cbo_subq_in.q.out        |    2 +
 .../clientpositive/tez/cbo_windowing.q.out      |   88 +-
 .../clientpositive/tez/constprog_dpp.q.out      |  113 +
 .../results/clientpositive/tez/cross_join.q.out |    8 +-
 .../test/results/clientpositive/tez/ctas.q.out  |    5 -
 .../tez/dynpart_sort_opt_vectorization.q.out    |   48 +-
 .../tez/dynpart_sort_optimization.q.out         |   32 -
 .../tez/dynpart_sort_optimization2.q.out        |   32 +-
 .../clientpositive/tez/explainuser_1.q.out      | 1999 ++---
 .../clientpositive/tez/explainuser_2.q.out      | 3438 ++++-----
 .../clientpositive/tez/explainuser_3.q.out      |  522 ++
 .../results/clientpositive/tez/groupby2.q.out   |   17 +-
 .../results/clientpositive/tez/having.q.out     |   28 +-
 .../results/clientpositive/tez/insert1.q.out    |  392 +-
 .../clientpositive/tez/limit_pushdown.q.out     |   78 +-
 .../clientpositive/tez/mapjoin_mapjoin.q.out    |  266 +-
 .../test/results/clientpositive/tez/mrr.q.out   |   82 +-
 .../clientpositive/tez/orc_analyze.q.out        |   48 -
 .../results/clientpositive/tez/orc_merge9.q.out |  186 +
 .../test/results/clientpositive/tez/ptf.q.out   |   28 +-
 .../clientpositive/tez/selectDistinctStar.q.out |    2 -
 .../results/clientpositive/tez/skewjoin.q.out   |   26 +-
 .../clientpositive/tez/stats_counter.q.out      |    2 -
 .../tez/stats_counter_partitioned.q.out         |   16 -
 .../clientpositive/tez/stats_noscan_1.q.out     |   17 -
 .../clientpositive/tez/stats_only_null.q.out    |    4 -
 .../tez/tez_dynpart_hashjoin_1.q.out            |  791 ++
 .../tez/tez_dynpart_hashjoin_2.q.out            |  564 ++
 .../results/clientpositive/tez/tez_fsstat.q.out |    2 -
 .../tez/tez_join_result_complex.q.out           | 2159 ++++++
 .../clientpositive/tez/tez_self_join.q.out      |  205 +
 .../results/clientpositive/tez/tez_smb_1.q.out  |  278 +
 .../results/clientpositive/tez/tez_union.q.out  |   88 +-
 .../tez/tez_union_dynamic_partition.q.out       |  156 +
 .../tez/tez_union_multiinsert.q.out             |    4 +-
 .../tez/tez_vector_dynpart_hashjoin_1.q.out     |  804 ++
 .../tez/tez_vector_dynpart_hashjoin_2.q.out     |  570 ++
 .../results/clientpositive/tez/union4.q.out     |    4 +
 .../results/clientpositive/tez/union6.q.out     |    4 +
 .../clientpositive/tez/unionDistinct_1.q.out    |   20 +-
 .../clientpositive/tez/update_all_types.q.out   |   48 +-
 .../clientpositive/tez/update_orig_table.q.out  |   17 +-
 .../clientpositive/tez/vector_coalesce.q.out    |  378 +-
 .../tez/vector_count_distinct.q.out             |    1 +
 .../clientpositive/tez/vector_decimal_2.q.out   |   55 +-
 .../tez/vector_decimal_expressions.q.out        |   99 +-
 .../tez/vector_decimal_round_2.q.out            |    8 +-
 .../tez/vector_groupby_reduce.q.out             |   10 +-
 .../clientpositive/tez/vector_interval_1.q.out  |   24 +-
 .../tez/vector_leftsemi_mapjoin.q.out           | 1032 +--
 .../tez/vector_multi_insert.q.out               |   20 +-
 .../clientpositive/tez/vector_outer_join1.q.out |  252 +-
 .../clientpositive/tez/vector_outer_join2.q.out |  122 +-
 .../clientpositive/tez/vector_outer_join3.q.out |  164 +-
 .../clientpositive/tez/vector_outer_join4.q.out | 1060 +--
 .../tez/vector_partitioned_date_time.q.out      |   10 +-
 .../clientpositive/tez/vectorization_0.q.out    |    8 +-
 .../clientpositive/tez/vectorization_1.q.out    |    8 +-
 .../clientpositive/tez/vectorization_10.q.out   |  428 +-
 .../clientpositive/tez/vectorization_11.q.out   |   74 +-
 .../clientpositive/tez/vectorization_12.q.out   |  994 +--
 .../clientpositive/tez/vectorization_13.q.out   |  414 +-
 .../clientpositive/tez/vectorization_14.q.out   | 1170 +--
 .../clientpositive/tez/vectorization_15.q.out   |   58 +-
 .../clientpositive/tez/vectorization_17.q.out   |  507 ++
 .../clientpositive/tez/vectorization_2.q.out    |    8 +-
 .../clientpositive/tez/vectorization_3.q.out    |    8 +-
 .../clientpositive/tez/vectorization_4.q.out    |    8 +-
 .../clientpositive/tez/vectorization_5.q.out    |    8 +-
 .../clientpositive/tez/vectorization_6.q.out    | 3042 ++++----
 .../clientpositive/tez/vectorization_7.q.out    |  320 +-
 .../clientpositive/tez/vectorization_8.q.out    |  306 +-
 .../clientpositive/tez/vectorization_9.q.out    |  957 ++-
 .../tez/vectorization_short_regress.q.out       | 1224 ++--
 .../clientpositive/tez/vectorized_casts.q.out   |   42 +-
 .../tez/vectorized_date_funcs.q.out             |  272 +-
 .../tez/vectorized_distinct_gby.q.out           |    1 +
 .../vectorized_dynamic_partition_pruning.q.out  |    8 +
 .../clientpositive/tez/vectorized_ptf.q.out     |  136 +-
 .../tez_join_result_complex.q.out               | 1337 ++++
 .../clientpositive/truncate_column.q.out        |   11 -
 .../udaf_percentile_approx_23.q.out             |   10 +-
 .../results/clientpositive/udtf_stack.q.out     |   12 +-
 .../clientpositive/unicode_notation.q.out       |    3 -
 .../test/results/clientpositive/union22.q.out   |    4 +-
 .../test/results/clientpositive/union28.q.out   |    4 +-
 .../test/results/clientpositive/union29.q.out   |    4 +-
 .../test/results/clientpositive/union30.q.out   |    4 +-
 .../test/results/clientpositive/union33.q.out   |    4 +-
 .../clientpositive/unionDistinct_1.q.out        |   20 +-
 .../clientpositive/union_date_trim.q.out        |    4 +-
 .../results/clientpositive/union_remove_1.q.out |    5 +-
 .../clientpositive/union_remove_10.q.out        |    3 +-
 .../clientpositive/union_remove_11.q.out        |    3 +-
 .../clientpositive/union_remove_12.q.out        |    3 +-
 .../clientpositive/union_remove_13.q.out        |    1 -
 .../clientpositive/union_remove_14.q.out        |    3 +-
 .../clientpositive/union_remove_15.q.out        |    9 +-
 .../clientpositive/union_remove_16.q.out        |    9 +-
 .../clientpositive/union_remove_17.q.out        |    5 +-
 .../clientpositive/union_remove_18.q.out        |   25 +-
 .../clientpositive/union_remove_19.q.out        |   13 +-
 .../results/clientpositive/union_remove_2.q.out |    3 +-
 .../clientpositive/union_remove_20.q.out        |    5 +-
 .../clientpositive/union_remove_21.q.out        |    3 +-
 .../clientpositive/union_remove_22.q.out        |   13 +-
 .../clientpositive/union_remove_23.q.out        |    3 +-
 .../clientpositive/union_remove_24.q.out        |    5 +-
 .../clientpositive/union_remove_25.q.out        |   22 +-
 .../results/clientpositive/union_remove_3.q.out |    3 +-
 .../results/clientpositive/union_remove_4.q.out |    5 +-
 .../results/clientpositive/union_remove_5.q.out |    3 +-
 .../results/clientpositive/union_remove_6.q.out |    8 +-
 .../clientpositive/union_remove_6_subq.q.out    |    8 +-
 .../results/clientpositive/union_remove_7.q.out |    5 +-
 .../results/clientpositive/union_remove_8.q.out |    3 +-
 .../results/clientpositive/union_remove_9.q.out |    3 +-
 .../clientpositive/union_top_level.q.out        |  316 +-
 .../clientpositive/update_all_types.q.out       |   48 +-
 .../clientpositive/update_orig_table.q.out      |   17 +-
 .../vector_aggregate_without_gby.q.out          |   96 +
 .../clientpositive/vector_coalesce.q.out        |  286 +-
 .../clientpositive/vector_decimal_2.q.out       |   55 +-
 .../vector_decimal_expressions.q.out            |   73 +-
 .../clientpositive/vector_decimal_round_2.q.out |    8 +-
 .../clientpositive/vector_groupby_reduce.q.out  |   10 +-
 .../clientpositive/vector_interval_1.q.out      |   24 +-
 .../clientpositive/vector_outer_join1.q.out     |  244 +-
 .../clientpositive/vector_outer_join2.q.out     |  118 +-
 .../clientpositive/vector_outer_join3.q.out     |  152 +-
 .../clientpositive/vector_outer_join4.q.out     | 1052 +--
 .../vector_partitioned_date_time.q.out          |   10 +-
 .../vector_partitioned_date_time_win.q.out      | 2036 ++++++
 .../clientpositive/vectorization_0.q.out        |    8 +-
 .../clientpositive/vectorization_1.q.out        |    8 +-
 .../clientpositive/vectorization_10.q.out       |  428 +-
 .../clientpositive/vectorization_11.q.out       |   74 +-
 .../clientpositive/vectorization_12.q.out       |  994 +--
 .../clientpositive/vectorization_13.q.out       |  414 +-
 .../clientpositive/vectorization_14.q.out       | 1170 +--
 .../clientpositive/vectorization_15.q.out       |   58 +-
 .../clientpositive/vectorization_17.q.out       |  500 ++
 .../clientpositive/vectorization_2.q.out        |    8 +-
 .../clientpositive/vectorization_3.q.out        |    8 +-
 .../clientpositive/vectorization_4.q.out        |    8 +-
 .../clientpositive/vectorization_5.q.out        |    8 +-
 .../clientpositive/vectorization_6.q.out        | 3042 ++++----
 .../clientpositive/vectorization_7.q.out        |  312 +-
 .../clientpositive/vectorization_8.q.out        |  298 +-
 .../clientpositive/vectorization_9.q.out        |  957 ++-
 .../vectorization_part_varchar.q.out            |   72 +
 .../vectorization_short_regress.q.out           | 1224 ++--
 .../clientpositive/vectorized_casts.q.out       |   42 +-
 .../clientpositive/vectorized_date_funcs.q.out  |  272 +-
 .../results/clientpositive/vectorized_ptf.q.out |  132 +-
 .../test/results/clientpositive/windowing.q.out |   44 +-
 .../clientpositive/windowing_expressions.q.out  |    8 +-
 .../clientpositive/windowing_windowspec3.q.out  |  232 +
 serde/pom.xml                                   |    4 +-
 .../hadoop/hive/ql/io/sarg/ExpressionTree.java  |  157 +
 .../hadoop/hive/ql/io/sarg/SearchArgument.java  |   14 +-
 .../hive/serde2/AbstractEncodingAwareSerDe.java |    7 +-
 .../hive/serde2/ColumnProjectionUtils.java      |   29 +-
 .../apache/hadoop/hive/serde2/SerDeUtils.java   |    4 +-
 .../hive/serde2/avro/AvroDeserializer.java      |   11 +-
 .../hadoop/hive/serde2/io/DateWritable.java     |   14 -
 .../hive/serde2/io/TimestampWritable.java       |    2 +-
 .../hive/serde2/lazybinary/LazyBinaryDate.java  |    4 +-
 .../hive/serde2/lazybinary/LazyBinarySerDe.java |   11 +-
 .../objectinspector/ObjectInspectorUtils.java   |   18 +-
 .../hive/serde2/avro/TestAvroDeserializer.java  |   63 +-
 .../avro/TestAvroObjectInspectorGenerator.java  |   33 +
 .../hive/serde2/io/TestTimestampWritable.java   |    2 +-
 .../TestObjectInspectorUtils.java               |   22 +
 service/pom.xml                                 |    2 +-
 .../auth/LdapAuthenticationProviderImpl.java    |  460 +-
 .../service/auth/TSetIpAddressProcessor.java    |    4 +-
 .../org/apache/hive/service/cli/CLIService.java |   13 -
 .../service/cli/operation/SQLOperation.java     |   16 +-
 .../hive/service/cli/session/HiveSession.java   |   12 +-
 .../service/cli/session/HiveSessionBase.java    |    2 -
 .../service/cli/session/HiveSessionImpl.java    |  100 +-
 .../cli/session/HiveSessionImplwithUGI.java     |   84 +-
 .../service/cli/session/SessionManager.java     |    6 +-
 .../service/cli/thrift/ThriftCLIService.java    |   25 +-
 .../service/cli/thrift/ThriftHttpServlet.java   |   18 +-
 .../apache/hive/service/server/HiveServer2.java |   15 +-
 .../apache/hive/service/cli/CLIServiceTest.java |  135 +-
 shims/0.20S/pom.xml                             |    2 +-
 .../apache/hadoop/mapred/WebHCatJTShim20S.java  |   10 +
 shims/0.23/pom.xml                              |    2 +-
 .../apache/hadoop/hive/shims/Hadoop23Shims.java |   11 +-
 .../apache/hadoop/mapred/WebHCatJTShim23.java   |   18 +-
 shims/aggregator/pom.xml                        |    2 +-
 shims/common/pom.xml                            |    2 +-
 .../apache/hadoop/hive/shims/HadoopShims.java   |    5 +
 .../hive/thrift/HadoopThriftAuthBridge.java     |    6 +-
 shims/pom.xml                                   |    2 +-
 shims/scheduler/pom.xml                         |    2 +-
 spark-client/pom.xml                            |    4 +-
 testutils/pom.xml                               |    2 +-
 1649 files changed, 95443 insertions(+), 36216 deletions(-)
----------------------------------------------------------------------



[25/50] [abbrv] hive git commit: HIVE-11080. Make VectorizedRowBatch.toString not depend on VectorExpressionWriter.

Posted by ga...@apache.org.
HIVE-11080. Make VectorizedRowBatch.toString not depend on
VectorExpressionWriter.


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

Branch: refs/heads/hbase-metastore
Commit: b6f48cb915b7e691b77cc4f328b9a32a082aa052
Parents: 178b8d1
Author: Owen O'Malley <om...@apache.org>
Authored: Thu Jun 25 11:21:09 2015 -0700
Committer: Owen O'Malley <om...@apache.org>
Committed: Fri Jul 17 09:43:52 2015 -0700

----------------------------------------------------------------------
 .../ql/exec/spark/SparkReduceRecordHandler.java |  2 -
 .../hive/ql/exec/tez/ReduceRecordSource.java    |  3 -
 .../hive/ql/exec/vector/BytesColumnVector.java  | 14 ++++
 .../hive/ql/exec/vector/ColumnVector.java       |  8 ++
 .../ql/exec/vector/DecimalColumnVector.java     | 12 +++
 .../hive/ql/exec/vector/DoubleColumnVector.java | 12 +++
 .../hive/ql/exec/vector/LongColumnVector.java   | 12 +++
 .../ql/exec/vector/VectorSelectOperator.java    |  3 -
 .../hive/ql/exec/vector/VectorizedRowBatch.java | 78 ++++++++------------
 9 files changed, 87 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/b6f48cb9/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkReduceRecordHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkReduceRecordHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkReduceRecordHandler.java
index ac5e3ca..bedccc3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkReduceRecordHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkReduceRecordHandler.java
@@ -364,8 +364,6 @@ public class SparkReduceRecordHandler extends SparkRecordHandler {
     } catch (Exception e) {
       String rowString = null;
       try {
-        /* batch.toString depends on this */
-        batch.setValueWriters(valueStringWriters[tag].toArray(new VectorExpressionWriter[0]));
         rowString = batch.toString();
       } catch (Exception e2) {
         rowString = "[Error getting row data with exception " + StringUtils.stringifyException(e2)

http://git-wip-us.apache.org/repos/asf/hive/blob/b6f48cb9/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 7d79e87..20f6dba 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
@@ -444,9 +444,6 @@ public class ReduceRecordSource implements RecordSource {
     } catch (Exception e) {
       String rowString = null;
       try {
-        /* batch.toString depends on this */
-        batch.setValueWriters(valueStringWriters
-            .toArray(new VectorExpressionWriter[0]));
         rowString = batch.toString();
       } catch (Exception e2) {
         rowString = "[Error getting row data with exception "

http://git-wip-us.apache.org/repos/asf/hive/blob/b6f48cb9/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 8ec7ead..c9a0fa2 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
@@ -330,4 +330,18 @@ public class BytesColumnVector extends ColumnVector {
   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/b6f48cb9/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 6654166..49d4c12 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
@@ -166,5 +166,13 @@ public abstract class ColumnVector {
     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/b6f48cb9/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 5009a42..0f63b29 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
@@ -85,6 +85,18 @@ public class DecimalColumnVector extends ColumnVector {
     }
   }
 
+  @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) {

http://git-wip-us.apache.org/repos/asf/hive/blob/b6f48cb9/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 525b3c5..013a9f5 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
@@ -146,4 +146,16 @@ public class DoubleColumnVector extends ColumnVector {
   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/b6f48cb9/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 f0545fe..d900cc6 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
@@ -190,4 +190,16 @@ public class LongColumnVector extends ColumnVector {
   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/b6f48cb9/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSelectOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSelectOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSelectOperator.java
index 212aa99..86b74e7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSelectOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSelectOperator.java
@@ -129,8 +129,6 @@ public class VectorSelectOperator extends Operator<SelectDesc> implements
     }
 
     // Prepare output, set the projections
-    VectorExpressionWriter [] originalValueWriters = vrg.valueWriters;
-    vrg.setValueWriters(valueWriters);
     int[] originalProjections = vrg.projectedColumns;
     int originalProjectionSize = vrg.projectionSize;
     vrg.projectionSize = vExpressions.length;
@@ -140,7 +138,6 @@ public class VectorSelectOperator extends Operator<SelectDesc> implements
     // Revert the projected columns back, because vrg will be re-used.
     vrg.projectionSize = originalProjectionSize;
     vrg.projectedColumns = originalProjections;
-    vrg.valueWriters = originalValueWriters;
   }
 
   static public String getOperatorName() {

http://git-wip-us.apache.org/repos/asf/hive/blob/b6f48cb9/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
index 7e41384..7c18da6 100644
--- 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
@@ -20,13 +20,7 @@ package org.apache.hadoop.hive.ql.exec.vector;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Map.Entry;
 
-import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpressionWriter;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.Writable;
 
@@ -61,8 +55,6 @@ public class VectorizedRowBatch implements Writable {
    */
   public static final int DEFAULT_SIZE = 1024;
 
-  public VectorExpressionWriter[] valueWriters = null;
-
   /**
    * Return a batch with the specified number of columns.
    * This is the standard constructor -- all batches should be the same size
@@ -112,7 +104,7 @@ public class VectorizedRowBatch implements Writable {
     return size;
   }
 
-  private String toUTF8(Object o) {
+  private static String toUTF8(Object o) {
     if(o == null || o instanceof NullWritable) {
       return "\\N"; /* as found in LazySimpleSerDe's nullSequence */
     }
@@ -125,47 +117,39 @@ public class VectorizedRowBatch implements Writable {
       return "";
     }
     StringBuilder b = new StringBuilder();
-    try {
-      if (this.selectedInUse) {
-        for (int j = 0; j < size; j++) {
-          int i = selected[j];
-          for (int k = 0; k < projectionSize; k++) {
-            int projIndex = projectedColumns[k];
-            ColumnVector cv = cols[projIndex];
-            if (k > 0) {
-              b.append('\u0001');
-            }
-            if (cv.isRepeating) {
-              b.append(toUTF8(valueWriters[k].writeValue(cv, 0)));
-            } else {
-              b.append(toUTF8(valueWriters[k].writeValue(cv, i)));
-            }
-          }
-          if (j < size - 1) {
-            b.append('\n');
+    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);
         }
-      } else {
-        for (int i = 0; i < size; i++) {
-          for (int k = 0; k < projectionSize; k++) {
-            int projIndex = projectedColumns[k];
-            ColumnVector cv = cols[projIndex];
-            if (k > 0) {
-              b.append('\u0001');
-            }
-            if (cv.isRepeating) {
-              b.append(toUTF8(valueWriters[k].writeValue(cv, 0)));
-            } else {
-              b.append(toUTF8(valueWriters[k].writeValue(cv, i)));
-            }
-          }
-          if (i < size - 1) {
-            b.append('\n');
+        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');
         }
       }
-    } catch (HiveException ex) {
-      throw new RuntimeException(ex);
     }
     return b.toString();
   }
@@ -180,10 +164,6 @@ public class VectorizedRowBatch implements Writable {
     throw new UnsupportedOperationException("Don't call me");
   }
 
-  public void setValueWriters(VectorExpressionWriter[] valueWriters) {
-    this.valueWriters = valueWriters;
-  }
-
   /**
    * Resets the row batch to default state
    *  - sets selectedInUse to false


[21/50] [abbrv] hive git commit: HIVE-11243: Changing log level in Utilities.getBaseWork(Nemon Lou, reviewed by Ferdinand Xu)

Posted by ga...@apache.org.
HIVE-11243: Changing log level in Utilities.getBaseWork(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/e2ee458d
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/e2ee458d
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/e2ee458d

Branch: refs/heads/hbase-metastore
Commit: e2ee458d8ef417785f1e9ebfca303a9d15fee8a8
Parents: e61a1a9
Author: Ferdinand Xu <ch...@intel.com>
Authored: Thu Jul 16 04:06:05 2015 -0400
Committer: Ferdinand Xu <ch...@intel.com>
Committed: Thu Jul 16 04:06:05 2015 -0400

----------------------------------------------------------------------
 ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/e2ee458d/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 afecb1e..d8e463d 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
@@ -453,7 +453,7 @@ public final class Utilities {
       return gWork;
     } catch (FileNotFoundException fnf) {
       // happens. e.g.: no reduce work.
-      LOG.info("File not found: " + fnf.getMessage());
+      LOG.debug("File not found: " + fnf.getMessage());
       LOG.info("No plan file found: "+path);
       return null;
     } catch (Exception e) {


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

Posted by ga...@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/hbase-metastore
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;


[02/50] [abbrv] hive git commit: HIVE-11239: Test encryption_insert_partition_static.q fails with different output results on other environments(Sergio Pena, reviewed by Ferdinand Xu)

Posted by ga...@apache.org.
HIVE-11239: Test encryption_insert_partition_static.q fails with different output results on other environments(Sergio Pena, 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/a65bcbdf
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/a65bcbdf
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/a65bcbdf

Branch: refs/heads/hbase-metastore
Commit: a65bcbdf463903a5a9650693d597a4b711abea2f
Parents: 21aecbc
Author: Ferdinand Xu <ch...@intel.com>
Authored: Tue Jul 14 04:50:15 2015 -0400
Committer: Ferdinand Xu <ch...@intel.com>
Committed: Tue Jul 14 04:50:15 2015 -0400

----------------------------------------------------------------------
 .../encryption_insert_partition_static.q        |  17 -
 .../encryption_insert_partition_static.q.out    | 739 +------------------
 2 files changed, 11 insertions(+), 745 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/a65bcbdf/ql/src/test/queries/clientpositive/encryption_insert_partition_static.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/encryption_insert_partition_static.q b/ql/src/test/queries/clientpositive/encryption_insert_partition_static.q
index c5769a6..69687df 100644
--- a/ql/src/test/queries/clientpositive/encryption_insert_partition_static.q
+++ b/ql/src/test/queries/clientpositive/encryption_insert_partition_static.q
@@ -18,11 +18,6 @@ create table unencryptedTable(key string,
     value string) partitioned by (ds string) clustered by (key) into 2 buckets stored as orc TBLPROPERTIES ('transactional'='true');
 
 -- insert encrypted table from values
-explain extended insert into table encryptedTable partition
-    (ds='today') values
-    ('501', 'val_501'),
-    ('502', 'val_502');
-
 insert into table encryptedTable partition
     (ds='today') values
     ('501', 'val_501'),
@@ -31,27 +26,15 @@ insert into table encryptedTable partition
 select * from encryptedTable order by key;
 
 -- insert encrypted table from unencrypted source
-explain extended
-insert into table encryptedTable partition (ds='yesterday')
-select * from src where key in ('238', '86');
-
 insert into table encryptedTable partition (ds='yesterday')
 select * from src where key in ('238', '86');
 
 select * from encryptedTable order by key;
 
 -- insert unencrypted table from encrypted source
-explain extended
 insert into table unencryptedTable partition (ds='today')
 select key, value from encryptedTable where ds='today';
 
-insert into table unencryptedTable partition (ds='today')
-select key, value from encryptedTable where ds='today';
-
-explain extended
-insert into table unencryptedTable partition (ds='yesterday')
-select key, value from encryptedTable where ds='yesterday';
-
 insert into table unencryptedTable partition (ds='yesterday')
 select key, value from encryptedTable where ds='yesterday';
 

http://git-wip-us.apache.org/repos/asf/hive/blob/a65bcbdf/ql/src/test/results/clientpositive/encrypted/encryption_insert_partition_static.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/encrypted/encryption_insert_partition_static.q.out b/ql/src/test/results/clientpositive/encrypted/encryption_insert_partition_static.q.out
index b10610c..c2f0ddc 100644
--- a/ql/src/test/results/clientpositive/encrypted/encryption_insert_partition_static.q.out
+++ b/ql/src/test/results/clientpositive/encrypted/encryption_insert_partition_static.q.out
@@ -39,187 +39,23 @@ POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@unencryptedTable
 PREHOOK: query: -- insert encrypted table from values
-explain extended insert into table encryptedTable partition
+insert into table encryptedTable partition
     (ds='today') values
     ('501', 'val_501'),
     ('502', 'val_502')
 PREHOOK: type: QUERY
-POSTHOOK: query: -- insert encrypted table from values
-explain extended insert into table encryptedTable partition
-    (ds='today') values
-    ('501', 'val_501'),
-    ('502', 'val_502')
-POSTHOOK: type: QUERY
-ABSTRACT SYNTAX TREE:
-  
-TOK_QUERY
-   TOK_FROM
-      null
-         null
-            Values__Tmp__Table__1
-   TOK_INSERT
-      TOK_INSERT_INTO
-         TOK_TAB
-            TOK_TABNAME
-               encryptedTable
-            TOK_PARTSPEC
-               TOK_PARTVAL
-                  ds
-                  'today'
-      TOK_SELECT
-         TOK_SELEXPR
-            TOK_ALLCOLREF
-
-
-STAGE DEPENDENCIES:
-  Stage-1 is a root stage
-  Stage-0 depends on stages: Stage-1
-  Stage-2 depends on stages: Stage-0
-
-STAGE PLANS:
-  Stage: Stage-1
-    Map Reduce
-      Map Operator Tree:
-          TableScan
-            alias: values__tmp__table__1
-            Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-            GatherStats: false
-            Select Operator
-              expressions: tmp_values_col1 (type: string), tmp_values_col2 (type: string)
-              outputColumnNames: _col0, _col1
-              Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-              Reduce Output Operator
-                sort order: 
-                Map-reduce partition columns: _col0 (type: string)
-                Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-                tag: -1
-                value expressions: _col0 (type: string), _col1 (type: string)
-                auto parallelism: false
-      Path -> Alias:
-#### A PARTIAL masked pattern was here #### data/warehouse/encryptedTable/.hive-staging
-      Path -> Partition:
-#### A PARTIAL masked pattern was here #### data/warehouse/encryptedTable/.hive-staging
-          Partition
-            base file name: Values__Tmp__Table__1
-            input format: org.apache.hadoop.mapred.TextInputFormat
-            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-            properties:
-              bucket_count -1
-              columns tmp_values_col1,tmp_values_col2
-              columns.comments 
-              columns.types string:string
-#### A masked pattern was here ####
-#### A PARTIAL masked pattern was here #### data/warehouse/encryptedTable/.hive-staging
-              name default.values__tmp__table__1
-              serialization.ddl struct values__tmp__table__1 { string tmp_values_col1, string tmp_values_col2}
-              serialization.format 1
-              serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-          
-              input format: org.apache.hadoop.mapred.TextInputFormat
-              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-              properties:
-                bucket_count -1
-                columns tmp_values_col1,tmp_values_col2
-                columns.comments 
-                columns.types string:string
-#### A masked pattern was here ####
-#### A PARTIAL masked pattern was here #### data/warehouse/encryptedTable/.hive-staging
-                name default.values__tmp__table__1
-                serialization.ddl struct values__tmp__table__1 { string tmp_values_col1, string tmp_values_col2}
-                serialization.format 1
-                serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-              name: default.values__tmp__table__1
-            name: default.values__tmp__table__1
-      Truncated Path -> Alias:
-#### A masked pattern was here ####
-      Needs Tagging: false
-      Reduce Operator Tree:
-        Select Operator
-          expressions: VALUE._col0 (type: string), VALUE._col1 (type: string)
-          outputColumnNames: _col0, _col1
-          Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-          File Output Operator
-            compressed: false
-            GlobalTableId: 1
-#### A PARTIAL masked pattern was here #### data/warehouse/encryptedTable/ds=today/.hive-staging
-            NumFilesPerFileSink: 1
-            Static Partition Specification: ds=today/
-            Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-#### A PARTIAL masked pattern was here #### data/warehouse/encryptedTable/ds=today/.hive-staging
-            table:
-                input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
-                output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
-                properties:
-                  bucket_count 2
-                  bucket_field_name key
-                  columns key,value
-                  columns.comments 
-                  columns.types string:string
-#### A masked pattern was here ####
-                  name default.encryptedtable
-                  partition_columns ds
-                  partition_columns.types string
-                  serialization.ddl struct encryptedtable { string key, string value}
-                  serialization.format 1
-                  serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-                  transactional true
-#### A masked pattern was here ####
-                serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
-                name: default.encryptedtable
-            TotalFiles: 1
-            GatherStats: true
-            MultiFileSpray: false
-
-  Stage: Stage-0
-    Move Operator
-      tables:
-          partition:
-            ds today
-          replace: false
-#### A PARTIAL masked pattern was here #### data/warehouse/encryptedTable/ds=today/.hive-staging
-          table:
-              input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
-              output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
-              properties:
-                bucket_count 2
-                bucket_field_name key
-                columns key,value
-                columns.comments 
-                columns.types string:string
-#### A masked pattern was here ####
-                name default.encryptedtable
-                partition_columns ds
-                partition_columns.types string
-                serialization.ddl struct encryptedtable { string key, string value}
-                serialization.format 1
-                serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-                transactional true
-#### A masked pattern was here ####
-              serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              name: default.encryptedtable
-
-  Stage: Stage-2
-    Stats-Aggr Operator
-#### A PARTIAL masked pattern was here #### data/warehouse/encryptedTable/ds=today/.hive-staging
-
-PREHOOK: query: insert into table encryptedTable partition
-    (ds='today') values
-    ('501', 'val_501'),
-    ('502', 'val_502')
-PREHOOK: type: QUERY
-PREHOOK: Input: default@values__tmp__table__2
+PREHOOK: Input: default@values__tmp__table__1
 PREHOOK: Output: default@encryptedtable@ds=today
-POSTHOOK: query: insert into table encryptedTable partition
+POSTHOOK: query: -- insert encrypted table from values
+insert into table encryptedTable partition
     (ds='today') values
     ('501', 'val_501'),
     ('502', 'val_502')
 POSTHOOK: type: QUERY
-POSTHOOK: Input: default@values__tmp__table__2
+POSTHOOK: Input: default@values__tmp__table__1
 POSTHOOK: Output: default@encryptedtable@ds=today
-POSTHOOK: Lineage: encryptedtable PARTITION(ds=today).key SIMPLE [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
-POSTHOOK: Lineage: encryptedtable PARTITION(ds=today).value SIMPLE [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+POSTHOOK: Lineage: encryptedtable PARTITION(ds=today).key SIMPLE [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+POSTHOOK: Lineage: encryptedtable PARTITION(ds=today).value SIMPLE [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
 PREHOOK: query: select * from encryptedTable order by key
 PREHOOK: type: QUERY
 PREHOOK: Input: default@encryptedtable
@@ -233,196 +69,13 @@ POSTHOOK: Input: default@encryptedtable@ds=today
 501	val_501	today
 502	val_502	today
 PREHOOK: query: -- insert encrypted table from unencrypted source
-explain extended
 insert into table encryptedTable partition (ds='yesterday')
 select * from src where key in ('238', '86')
 PREHOOK: type: QUERY
-POSTHOOK: query: -- insert encrypted table from unencrypted source
-explain extended
-insert into table encryptedTable partition (ds='yesterday')
-select * from src where key in ('238', '86')
-POSTHOOK: type: QUERY
-ABSTRACT SYNTAX TREE:
-  
-TOK_QUERY
-   TOK_FROM
-      TOK_TABREF
-         TOK_TABNAME
-            src
-   TOK_INSERT
-      TOK_INSERT_INTO
-         TOK_TAB
-            TOK_TABNAME
-               encryptedTable
-            TOK_PARTSPEC
-               TOK_PARTVAL
-                  ds
-                  'yesterday'
-      TOK_SELECT
-         TOK_SELEXPR
-            TOK_ALLCOLREF
-      TOK_WHERE
-         TOK_FUNCTION
-            in
-            TOK_TABLE_OR_COL
-               key
-            '238'
-            '86'
-
-
-STAGE DEPENDENCIES:
-  Stage-1 is a root stage
-  Stage-0 depends on stages: Stage-1
-  Stage-2 depends on stages: Stage-0
-
-STAGE PLANS:
-  Stage: Stage-1
-    Map Reduce
-      Map Operator Tree:
-          TableScan
-            alias: src
-            Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
-            GatherStats: false
-            Filter Operator
-              isSamplingPred: false
-              predicate: (key) IN ('238', '86') (type: boolean)
-              Statistics: Num rows: 14 Data size: 2805 Basic stats: COMPLETE Column stats: NONE
-              Select Operator
-                expressions: key (type: string), value (type: string)
-                outputColumnNames: _col0, _col1
-                Statistics: Num rows: 14 Data size: 2805 Basic stats: COMPLETE Column stats: NONE
-                Reduce Output Operator
-                  sort order: 
-                  Map-reduce partition columns: _col0 (type: string)
-                  Statistics: Num rows: 14 Data size: 2805 Basic stats: COMPLETE Column stats: NONE
-                  tag: -1
-                  value expressions: _col0 (type: string), _col1 (type: string)
-                  auto parallelism: false
-      Path -> Alias:
-#### A masked pattern was here ####
-      Path -> Partition:
-#### A masked pattern was here ####
-          Partition
-            base file name: src
-            input format: org.apache.hadoop.mapred.TextInputFormat
-            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-            properties:
-              COLUMN_STATS_ACCURATE true
-              bucket_count -1
-              columns key,value
-              columns.comments 'default','default'
-              columns.types string:string
-#### A masked pattern was here ####
-              name default.src
-              numFiles 1
-              numRows 0
-              rawDataSize 0
-              serialization.ddl struct src { string key, string value}
-              serialization.format 1
-              serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-              totalSize 5812
-#### A masked pattern was here ####
-            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-          
-              input format: org.apache.hadoop.mapred.TextInputFormat
-              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-              properties:
-                COLUMN_STATS_ACCURATE true
-                bucket_count -1
-                columns key,value
-                columns.comments 'default','default'
-                columns.types string:string
-#### A masked pattern was here ####
-                name default.src
-                numFiles 1
-                numRows 0
-                rawDataSize 0
-                serialization.ddl struct src { string key, string value}
-                serialization.format 1
-                serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                totalSize 5812
-#### A masked pattern was here ####
-              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-              name: default.src
-            name: default.src
-      Truncated Path -> Alias:
-        /src [src]
-      Needs Tagging: false
-      Reduce Operator Tree:
-        Select Operator
-          expressions: VALUE._col0 (type: string), VALUE._col1 (type: string)
-          outputColumnNames: _col0, _col1
-          Statistics: Num rows: 14 Data size: 2805 Basic stats: COMPLETE Column stats: NONE
-          File Output Operator
-            compressed: false
-            GlobalTableId: 1
-#### A PARTIAL masked pattern was here #### data/warehouse/encryptedTable/ds=yesterday/.hive-staging
-            NumFilesPerFileSink: 1
-            Static Partition Specification: ds=yesterday/
-            Statistics: Num rows: 14 Data size: 2805 Basic stats: COMPLETE Column stats: NONE
-#### A PARTIAL masked pattern was here #### data/warehouse/encryptedTable/ds=yesterday/.hive-staging
-            table:
-                input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
-                output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
-                properties:
-                  bucket_count 2
-                  bucket_field_name key
-                  columns key,value
-                  columns.comments 
-                  columns.types string:string
-#### A masked pattern was here ####
-                  name default.encryptedtable
-                  partition_columns ds
-                  partition_columns.types string
-                  serialization.ddl struct encryptedtable { string key, string value}
-                  serialization.format 1
-                  serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-                  transactional true
-#### A masked pattern was here ####
-                serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
-                name: default.encryptedtable
-            TotalFiles: 1
-            GatherStats: true
-            MultiFileSpray: false
-
-  Stage: Stage-0
-    Move Operator
-      tables:
-          partition:
-            ds yesterday
-          replace: false
-#### A PARTIAL masked pattern was here #### data/warehouse/encryptedTable/ds=yesterday/.hive-staging
-          table:
-              input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
-              output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
-              properties:
-                bucket_count 2
-                bucket_field_name key
-                columns key,value
-                columns.comments 
-                columns.types string:string
-#### A masked pattern was here ####
-                name default.encryptedtable
-                partition_columns ds
-                partition_columns.types string
-                serialization.ddl struct encryptedtable { string key, string value}
-                serialization.format 1
-                serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-                transactional true
-#### A masked pattern was here ####
-              serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              name: default.encryptedtable
-
-  Stage: Stage-2
-    Stats-Aggr Operator
-#### A PARTIAL masked pattern was here #### data/warehouse/encryptedTable/ds=yesterday/.hive-staging
-
-PREHOOK: query: insert into table encryptedTable partition (ds='yesterday')
-select * from src where key in ('238', '86')
-PREHOOK: type: QUERY
 PREHOOK: Input: default@src
 PREHOOK: Output: default@encryptedtable@ds=yesterday
-POSTHOOK: query: insert into table encryptedTable partition (ds='yesterday')
+POSTHOOK: query: -- insert encrypted table from unencrypted source
+insert into table encryptedTable partition (ds='yesterday')
 select * from src where key in ('238', '86')
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src
@@ -447,199 +100,14 @@ POSTHOOK: Input: default@encryptedtable@ds=yesterday
 502	val_502	today
 86	val_86	yesterday
 PREHOOK: query: -- insert unencrypted table from encrypted source
-explain extended
 insert into table unencryptedTable partition (ds='today')
 select key, value from encryptedTable where ds='today'
 PREHOOK: type: QUERY
-POSTHOOK: query: -- insert unencrypted table from encrypted source
-explain extended
-insert into table unencryptedTable partition (ds='today')
-select key, value from encryptedTable where ds='today'
-POSTHOOK: type: QUERY
-ABSTRACT SYNTAX TREE:
-  
-TOK_QUERY
-   TOK_FROM
-      TOK_TABREF
-         TOK_TABNAME
-            encryptedTable
-   TOK_INSERT
-      TOK_INSERT_INTO
-         TOK_TAB
-            TOK_TABNAME
-               unencryptedTable
-            TOK_PARTSPEC
-               TOK_PARTVAL
-                  ds
-                  'today'
-      TOK_SELECT
-         TOK_SELEXPR
-            TOK_TABLE_OR_COL
-               key
-         TOK_SELEXPR
-            TOK_TABLE_OR_COL
-               value
-      TOK_WHERE
-         =
-            TOK_TABLE_OR_COL
-               ds
-            'today'
-
-
-STAGE DEPENDENCIES:
-  Stage-1 is a root stage
-  Stage-0 depends on stages: Stage-1
-  Stage-2 depends on stages: Stage-0
-
-STAGE PLANS:
-  Stage: Stage-1
-    Map Reduce
-      Map Operator Tree:
-          TableScan
-            alias: encryptedtable
-            Statistics: Num rows: 6 Data size: 1351 Basic stats: COMPLETE Column stats: NONE
-            GatherStats: false
-            Select Operator
-              expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
-              Statistics: Num rows: 6 Data size: 1351 Basic stats: COMPLETE Column stats: NONE
-              Reduce Output Operator
-                sort order: 
-                Map-reduce partition columns: _col0 (type: string)
-                Statistics: Num rows: 6 Data size: 1351 Basic stats: COMPLETE Column stats: NONE
-                tag: -1
-                value expressions: _col0 (type: string), _col1 (type: string)
-                auto parallelism: false
-      Path -> Alias:
-#### A masked pattern was here ####
-      Path -> Partition:
-#### A masked pattern was here ####
-          Partition
-            base file name: ds=today
-            input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
-            output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
-            partition values:
-              ds today
-            properties:
-              COLUMN_STATS_ACCURATE true
-              bucket_count 2
-              bucket_field_name key
-              columns key,value
-              columns.comments 
-              columns.types string:string
-#### A masked pattern was here ####
-              name default.encryptedtable
-              numFiles 2
-              numRows 0
-              partition_columns ds
-              partition_columns.types string
-              rawDataSize 0
-              serialization.ddl struct encryptedtable { string key, string value}
-              serialization.format 1
-              serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 1351
-#### A masked pattern was here ####
-            serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
-          
-              input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
-              output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
-              properties:
-                bucket_count 2
-                bucket_field_name key
-                columns key,value
-                columns.comments 
-                columns.types string:string
-#### A masked pattern was here ####
-                name default.encryptedtable
-                partition_columns ds
-                partition_columns.types string
-                serialization.ddl struct encryptedtable { string key, string value}
-                serialization.format 1
-                serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-                transactional true
-#### A masked pattern was here ####
-              serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              name: default.encryptedtable
-            name: default.encryptedtable
-      Truncated Path -> Alias:
-        /encryptedTable/ds=today [encryptedtable]
-      Needs Tagging: false
-      Reduce Operator Tree:
-        Select Operator
-          expressions: VALUE._col0 (type: string), VALUE._col1 (type: string)
-          outputColumnNames: _col0, _col1
-          Statistics: Num rows: 6 Data size: 1351 Basic stats: COMPLETE Column stats: NONE
-          File Output Operator
-            compressed: false
-            GlobalTableId: 1
-#### A PARTIAL masked pattern was here #### data/warehouse/unencryptedtable/ds=today/.hive-staging
-            NumFilesPerFileSink: 1
-            Static Partition Specification: ds=today/
-            Statistics: Num rows: 6 Data size: 1351 Basic stats: COMPLETE Column stats: NONE
-#### A PARTIAL masked pattern was here #### data/warehouse/unencryptedtable/ds=today/.hive-staging
-            table:
-                input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
-                output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
-                properties:
-                  bucket_count 2
-                  bucket_field_name key
-                  columns key,value
-                  columns.comments 
-                  columns.types string:string
-#### A masked pattern was here ####
-                  name default.unencryptedtable
-                  partition_columns ds
-                  partition_columns.types string
-                  serialization.ddl struct unencryptedtable { string key, string value}
-                  serialization.format 1
-                  serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-                  transactional true
-#### A masked pattern was here ####
-                serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
-                name: default.unencryptedtable
-            TotalFiles: 1
-            GatherStats: true
-            MultiFileSpray: false
-
-  Stage: Stage-0
-    Move Operator
-      tables:
-          partition:
-            ds today
-          replace: false
-#### A PARTIAL masked pattern was here #### data/warehouse/unencryptedtable/ds=today/.hive-staging
-          table:
-              input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
-              output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
-              properties:
-                bucket_count 2
-                bucket_field_name key
-                columns key,value
-                columns.comments 
-                columns.types string:string
-#### A masked pattern was here ####
-                name default.unencryptedtable
-                partition_columns ds
-                partition_columns.types string
-                serialization.ddl struct unencryptedtable { string key, string value}
-                serialization.format 1
-                serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-                transactional true
-#### A masked pattern was here ####
-              serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              name: default.unencryptedtable
-
-  Stage: Stage-2
-    Stats-Aggr Operator
-#### A PARTIAL masked pattern was here #### data/warehouse/unencryptedtable/ds=today/.hive-staging
-
-PREHOOK: query: insert into table unencryptedTable partition (ds='today')
-select key, value from encryptedTable where ds='today'
-PREHOOK: type: QUERY
 PREHOOK: Input: default@encryptedtable
 PREHOOK: Input: default@encryptedtable@ds=today
 PREHOOK: Output: default@unencryptedtable@ds=today
-POSTHOOK: query: insert into table unencryptedTable partition (ds='today')
+POSTHOOK: query: -- insert unencrypted table from encrypted source
+insert into table unencryptedTable partition (ds='today')
 select key, value from encryptedTable where ds='today'
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@encryptedtable
@@ -647,191 +115,6 @@ POSTHOOK: Input: default@encryptedtable@ds=today
 POSTHOOK: Output: default@unencryptedtable@ds=today
 POSTHOOK: Lineage: unencryptedtable PARTITION(ds=today).key SIMPLE [(encryptedtable)encryptedtable.FieldSchema(name:key, type:string, comment:null), ]
 POSTHOOK: Lineage: unencryptedtable PARTITION(ds=today).value SIMPLE [(encryptedtable)encryptedtable.FieldSchema(name:value, type:string, comment:null), ]
-PREHOOK: query: explain extended
-insert into table unencryptedTable partition (ds='yesterday')
-select key, value from encryptedTable where ds='yesterday'
-PREHOOK: type: QUERY
-POSTHOOK: query: explain extended
-insert into table unencryptedTable partition (ds='yesterday')
-select key, value from encryptedTable where ds='yesterday'
-POSTHOOK: type: QUERY
-ABSTRACT SYNTAX TREE:
-  
-TOK_QUERY
-   TOK_FROM
-      TOK_TABREF
-         TOK_TABNAME
-            encryptedTable
-   TOK_INSERT
-      TOK_INSERT_INTO
-         TOK_TAB
-            TOK_TABNAME
-               unencryptedTable
-            TOK_PARTSPEC
-               TOK_PARTVAL
-                  ds
-                  'yesterday'
-      TOK_SELECT
-         TOK_SELEXPR
-            TOK_TABLE_OR_COL
-               key
-         TOK_SELEXPR
-            TOK_TABLE_OR_COL
-               value
-      TOK_WHERE
-         =
-            TOK_TABLE_OR_COL
-               ds
-            'yesterday'
-
-
-STAGE DEPENDENCIES:
-  Stage-1 is a root stage
-  Stage-0 depends on stages: Stage-1
-  Stage-2 depends on stages: Stage-0
-
-STAGE PLANS:
-  Stage: Stage-1
-    Map Reduce
-      Map Operator Tree:
-          TableScan
-            alias: encryptedtable
-            Statistics: Num rows: 6 Data size: 1372 Basic stats: COMPLETE Column stats: NONE
-            GatherStats: false
-            Select Operator
-              expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
-              Statistics: Num rows: 6 Data size: 1372 Basic stats: COMPLETE Column stats: NONE
-              Reduce Output Operator
-                sort order: 
-                Map-reduce partition columns: _col0 (type: string)
-                Statistics: Num rows: 6 Data size: 1372 Basic stats: COMPLETE Column stats: NONE
-                tag: -1
-                value expressions: _col0 (type: string), _col1 (type: string)
-                auto parallelism: false
-      Path -> Alias:
-#### A masked pattern was here ####
-      Path -> Partition:
-#### A masked pattern was here ####
-          Partition
-            base file name: ds=yesterday
-            input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
-            output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
-            partition values:
-              ds yesterday
-            properties:
-              COLUMN_STATS_ACCURATE true
-              bucket_count 2
-              bucket_field_name key
-              columns key,value
-              columns.comments 
-              columns.types string:string
-#### A masked pattern was here ####
-              name default.encryptedtable
-              numFiles 2
-              numRows 0
-              partition_columns ds
-              partition_columns.types string
-              rawDataSize 0
-              serialization.ddl struct encryptedtable { string key, string value}
-              serialization.format 1
-              serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 1372
-#### A masked pattern was here ####
-            serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
-          
-              input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
-              output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
-              properties:
-                bucket_count 2
-                bucket_field_name key
-                columns key,value
-                columns.comments 
-                columns.types string:string
-#### A masked pattern was here ####
-                name default.encryptedtable
-                partition_columns ds
-                partition_columns.types string
-                serialization.ddl struct encryptedtable { string key, string value}
-                serialization.format 1
-                serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-                transactional true
-#### A masked pattern was here ####
-              serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              name: default.encryptedtable
-            name: default.encryptedtable
-      Truncated Path -> Alias:
-        /encryptedTable/ds=yesterday [encryptedtable]
-      Needs Tagging: false
-      Reduce Operator Tree:
-        Select Operator
-          expressions: VALUE._col0 (type: string), VALUE._col1 (type: string)
-          outputColumnNames: _col0, _col1
-          Statistics: Num rows: 6 Data size: 1372 Basic stats: COMPLETE Column stats: NONE
-          File Output Operator
-            compressed: false
-            GlobalTableId: 1
-#### A PARTIAL masked pattern was here #### data/warehouse/unencryptedtable/ds=yesterday/.hive-staging
-            NumFilesPerFileSink: 1
-            Static Partition Specification: ds=yesterday/
-            Statistics: Num rows: 6 Data size: 1372 Basic stats: COMPLETE Column stats: NONE
-#### A PARTIAL masked pattern was here #### data/warehouse/unencryptedtable/ds=yesterday/.hive-staging
-            table:
-                input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
-                output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
-                properties:
-                  bucket_count 2
-                  bucket_field_name key
-                  columns key,value
-                  columns.comments 
-                  columns.types string:string
-#### A masked pattern was here ####
-                  name default.unencryptedtable
-                  partition_columns ds
-                  partition_columns.types string
-                  serialization.ddl struct unencryptedtable { string key, string value}
-                  serialization.format 1
-                  serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-                  transactional true
-#### A masked pattern was here ####
-                serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
-                name: default.unencryptedtable
-            TotalFiles: 1
-            GatherStats: true
-            MultiFileSpray: false
-
-  Stage: Stage-0
-    Move Operator
-      tables:
-          partition:
-            ds yesterday
-          replace: false
-#### A PARTIAL masked pattern was here #### data/warehouse/unencryptedtable/ds=yesterday/.hive-staging
-          table:
-              input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
-              output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
-              properties:
-                bucket_count 2
-                bucket_field_name key
-                columns key,value
-                columns.comments 
-                columns.types string:string
-#### A masked pattern was here ####
-                name default.unencryptedtable
-                partition_columns ds
-                partition_columns.types string
-                serialization.ddl struct unencryptedtable { string key, string value}
-                serialization.format 1
-                serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-                transactional true
-#### A masked pattern was here ####
-              serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              name: default.unencryptedtable
-
-  Stage: Stage-2
-    Stats-Aggr Operator
-#### A PARTIAL masked pattern was here #### data/warehouse/unencryptedtable/ds=yesterday/.hive-staging
-
 PREHOOK: query: insert into table unencryptedTable partition (ds='yesterday')
 select key, value from encryptedTable where ds='yesterday'
 PREHOOK: type: QUERY


[46/50] [abbrv] 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 ga...@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/hbase-metastore
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++) {


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

Posted by ga...@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/hbase-metastore
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 {


[32/50] [abbrv] hive git commit: HIVE-11291 : Avoid allocation storm while doing rule matching on operator/expression trees (Ashutosh Chauhan via Hari Sankar)

Posted by ga...@apache.org.
HIVE-11291 : Avoid allocation storm while doing rule matching on operator/expression trees (Ashutosh Chauhan via Hari Sankar)


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

Branch: refs/heads/hbase-metastore
Commit: 7b17df1501a76544a750f36a7afa871038f40109
Parents: 8646c12
Author: Ashutosh Chauhan <ha...@apache.org>
Authored: Fri Jul 17 14:30:22 2015 -0700
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Fri Jul 17 14:30:22 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hive/ql/lib/RuleExactMatch.java      | 21 ++++++++++----------
 .../hadoop/hive/ql/optimizer/PrunerUtils.java   | 14 ++++++-------
 .../BucketingSortingInferenceOptimizer.java     |  8 ++++----
 .../hive/ql/ppd/ExprWalkerProcFactory.java      | 17 +++++++---------
 4 files changed, 29 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/7b17df15/ql/src/java/org/apache/hadoop/hive/ql/lib/RuleExactMatch.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lib/RuleExactMatch.java b/ql/src/java/org/apache/hadoop/hive/ql/lib/RuleExactMatch.java
index 5e5c054..6f7962e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lib/RuleExactMatch.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lib/RuleExactMatch.java
@@ -30,7 +30,7 @@ import org.apache.hadoop.hive.ql.parse.SemanticException;
 public class RuleExactMatch implements Rule {
 
   private final String ruleName;
-  private final String pattern;
+  private final String[] pattern;
 
   /**
    * The rule specified as operator names separated by % symbols, the left side represents the
@@ -45,7 +45,7 @@ public class RuleExactMatch implements Rule {
    * @param regExp
    *          string specification of the rule
    **/
-  public RuleExactMatch(String ruleName, String pattern) {
+  public RuleExactMatch(String ruleName, String[] pattern) {
     this.ruleName = ruleName;
     this.pattern = pattern;
   }
@@ -62,23 +62,24 @@ public class RuleExactMatch implements Rule {
    * @return cost of the function
    * @throws SemanticException
    */
+  @Override
   public int cost(Stack<Node> stack) throws SemanticException {
     int numElems = (stack != null ? stack.size() : 0);
-    String name = new String();
-    for (int pos = numElems - 1; pos >= 0; pos--) {
-      name = stack.get(pos).getName() + "%" + name;
+    if (numElems != pattern.length) {
+      return -1;
     }
-
-    if (pattern.equals(name)) {
-      return 1;
+    for (int pos = numElems - 1; pos >= 0; pos--) {
+      if(!stack.get(pos).getName().equals(pattern[pos])) {
+        return -1;
+      }
     }
-
-    return -1;
+    return numElems;
   }
 
   /**
    * @return the name of the Node
    **/
+  @Override
   public String getName() {
     return ruleName;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/7b17df15/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PrunerUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PrunerUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PrunerUtils.java
index 108177e..5d375f6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PrunerUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/PrunerUtils.java
@@ -35,7 +35,9 @@ import org.apache.hadoop.hive.ql.lib.Node;
 import org.apache.hadoop.hive.ql.lib.NodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.RuleExactMatch;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
+import org.apache.hadoop.hive.ql.lib.TypeRule;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
@@ -76,9 +78,8 @@ public final class PrunerUtils {
     String tsOprName = TableScanOperator.getOperatorName();
     String filtOprName = FilterOperator.getOperatorName();
 
-    opRules.put(new RuleRegExp("R1", new StringBuilder().append("(").append(tsOprName).append("%")
-        .append(filtOprName).append("%)|(").append(tsOprName).append("%").append(filtOprName)
-        .append("%").append(filtOprName).append("%)").toString()), filterProc);
+    opRules.put(new RuleExactMatch("R1", new String[] {tsOprName, filtOprName, filtOprName}), filterProc);
+    opRules.put(new RuleExactMatch("R2", new String[] {tsOprName, filtOprName}), filterProc);
 
     // The dispatcher fires the processor corresponding to the closest matching
     // rule and passes the context along
@@ -111,10 +112,9 @@ public final class PrunerUtils {
     // the operator stack. The dispatcher
     // generates the plan from the operator tree
     Map<Rule, NodeProcessor> exprRules = new LinkedHashMap<Rule, NodeProcessor>();
-    exprRules.put(new RuleRegExp("R1", ExprNodeColumnDesc.class.getName() + "%"), colProc);
-    exprRules.put(new RuleRegExp("R2", ExprNodeFieldDesc.class.getName() + "%"), fieldProc);
-    exprRules.put(new RuleRegExp("R5", ExprNodeGenericFuncDesc.class.getName() + "%"),
-        genFuncProc);
+    exprRules.put(new TypeRule(ExprNodeColumnDesc.class) , colProc);
+    exprRules.put(new TypeRule(ExprNodeFieldDesc.class), fieldProc);
+    exprRules.put(new TypeRule(ExprNodeGenericFuncDesc.class), genFuncProc);
 
     // The dispatcher fires the processor corresponding to the closest matching
     // rule and passes the context along

http://git-wip-us.apache.org/repos/asf/hive/blob/7b17df15/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/BucketingSortingInferenceOptimizer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/BucketingSortingInferenceOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/BucketingSortingInferenceOptimizer.java
index f370d4d..a6b8d54 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/BucketingSortingInferenceOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/BucketingSortingInferenceOptimizer.java
@@ -104,10 +104,10 @@ public class BucketingSortingInferenceOptimizer implements PhysicalPlanResolver
           BucketingSortingOpProcFactory.getSelProc());
       // Matches only GroupByOperators which are reducers, rather than map group by operators,
       // or multi group by optimization specific operators
-      opRules.put(new RuleExactMatch("R2", GroupByOperator.getOperatorName() + "%"),
+      opRules.put(new RuleExactMatch("R2", new String[]{GroupByOperator.getOperatorName()}),
           BucketingSortingOpProcFactory.getGroupByProc());
       // Matches only JoinOperators which are reducers, rather than map joins, SMB map joins, etc.
-      opRules.put(new RuleExactMatch("R3", JoinOperator.getOperatorName() + "%"),
+      opRules.put(new RuleExactMatch("R3", new String[]{JoinOperator.getOperatorName()}),
           BucketingSortingOpProcFactory.getJoinProc());
       opRules.put(new RuleRegExp("R5", FileSinkOperator.getOperatorName() + "%"),
           BucketingSortingOpProcFactory.getFileSinkProc());
@@ -126,8 +126,8 @@ public class BucketingSortingInferenceOptimizer implements PhysicalPlanResolver
           BucketingSortingOpProcFactory.getForwardProc());
       // Matches only ForwardOperators which are reducers and are followed by GroupByOperators
       // (specific to the multi group by optimization)
-      opRules.put(new RuleExactMatch("R12", ForwardOperator.getOperatorName() + "%" +
-          GroupByOperator.getOperatorName() + "%"),
+      opRules.put(new RuleExactMatch("R12",new String[]{ ForwardOperator.getOperatorName(),
+          GroupByOperator.getOperatorName()}),
           BucketingSortingOpProcFactory.getMultiGroupByProc());
 
       // The dispatcher fires the processor corresponding to the closest matching rule and passes

http://git-wip-us.apache.org/repos/asf/hive/blob/7b17df15/ql/src/java/org/apache/hadoop/hive/ql/ppd/ExprWalkerProcFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ppd/ExprWalkerProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/ppd/ExprWalkerProcFactory.java
index 3a07b17..6a1bef9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ppd/ExprWalkerProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ppd/ExprWalkerProcFactory.java
@@ -38,7 +38,9 @@ import org.apache.hadoop.hive.ql.lib.Node;
 import org.apache.hadoop.hive.ql.lib.NodeProcessor;
 import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.lib.Rule;
+import org.apache.hadoop.hive.ql.lib.RuleExactMatch;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
+import org.apache.hadoop.hive.ql.lib.TypeRule;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
@@ -267,14 +269,9 @@ public final class ExprWalkerProcFactory {
     // the operator stack. The dispatcher
     // generates the plan from the operator tree
     Map<Rule, NodeProcessor> exprRules = new LinkedHashMap<Rule, NodeProcessor>();
-    exprRules.put(
-        new RuleRegExp("R1", ExprNodeColumnDesc.class.getName() + "%"),
-        getColumnProcessor());
-    exprRules.put(
-        new RuleRegExp("R2", ExprNodeFieldDesc.class.getName() + "%"),
-        getFieldProcessor());
-    exprRules.put(new RuleRegExp("R3", ExprNodeGenericFuncDesc.class.getName()
-        + "%"), getGenericFuncProcessor());
+    exprRules.put(new TypeRule(ExprNodeColumnDesc.class), getColumnProcessor());
+    exprRules.put(new TypeRule(ExprNodeFieldDesc.class), getFieldProcessor());
+    exprRules.put(new TypeRule(ExprNodeGenericFuncDesc.class), getGenericFuncProcessor());
 
     // The dispatcher fires the processor corresponding to the closest matching
     // rule and passes the context along
@@ -319,9 +316,9 @@ public final class ExprWalkerProcFactory {
       assert ctx.getNewToOldExprMap().containsKey(expr);
       for (int i = 0; i < expr.getChildren().size(); i++) {
         ctx.getNewToOldExprMap().put(
-            (ExprNodeDesc) expr.getChildren().get(i),
+            expr.getChildren().get(i),
             ctx.getNewToOldExprMap().get(expr).getChildren().get(i));
-        extractFinalCandidates((ExprNodeDesc) expr.getChildren().get(i),
+        extractFinalCandidates(expr.getChildren().get(i),
             ctx, conf);
       }
       return;


[24/50] [abbrv] hive git commit: HIVE-11255 - get_table_objects_by_name() in HiveMetaStore.java needs to retrieve table objects in multiple batches (Aihua Xu, reviewed by Chao Sun)

Posted by ga...@apache.org.
HIVE-11255 - get_table_objects_by_name() in HiveMetaStore.java needs to retrieve table objects in multiple batches (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/178b8d17
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/178b8d17
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/178b8d17

Branch: refs/heads/hbase-metastore
Commit: 178b8d17fcaa5293dbe75eff5d39871a47f51c81
Parents: 854950b
Author: Aihua Xu <ai...@gmail.com>
Authored: Thu Jul 16 15:03:40 2015 -0700
Committer: Chao Sun <su...@apache.org>
Committed: Thu Jul 16 15:03:40 2015 -0700

----------------------------------------------------------------------
 .../hive/metastore/TestHiveMetaStore.java       | 57 +++++++++++++++++++-
 .../hadoop/hive/metastore/HiveMetaStore.java    | 48 +++++++++++++----
 2 files changed, 93 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/178b8d17/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
index dffeb34..06c6b76 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData;
@@ -99,6 +100,8 @@ public abstract class TestHiveMetaStore extends TestCase {
     hiveConf.set("hive.key2", "http://www.example.com");
     hiveConf.set("hive.key3", "");
     hiveConf.set("hive.key4", "0");
+
+    hiveConf.setIntVar(ConfVars.METASTORE_BATCH_RETRIEVE_MAX, 2);
   }
 
   public void testNameMethods() {
@@ -1330,7 +1333,7 @@ public abstract class TestHiveMetaStore extends TestCase {
       tableNames.add(tblName2);
       List<Table> foundTables = client.getTableObjectsByName(dbName, tableNames);
 
-      assertEquals(foundTables.size(), 2);
+      assertEquals(2, foundTables.size());
       for (Table t: foundTables) {
         if (t.getTableName().equals(tblName2)) {
           assertEquals(t.getSd().getLocation(), tbl2.getSd().getLocation());
@@ -2700,6 +2703,26 @@ public abstract class TestHiveMetaStore extends TestCase {
     return typ1;
   }
 
+  /**
+   * Creates a simple table under specified database
+   * @param dbName    the database name that the table will be created under
+   * @param tableName the table name to be created
+   * @throws Exception
+   */
+  private void createTable(String dbName, String tableName)
+      throws Exception {
+    List<FieldSchema> columns = new ArrayList<FieldSchema>();
+    columns.add(new FieldSchema("foo", "string", ""));
+    columns.add(new FieldSchema("bar", "string", ""));
+
+    Map<String, String> serdParams = new HashMap<String, String>();
+    serdParams.put(serdeConstants.SERIALIZATION_FORMAT, "1");
+
+    StorageDescriptor sd =  createStorageDescriptor(tableName, columns, null, serdParams);
+
+    createTable(dbName, tableName, null, null, null, sd, 0);
+  }
+
   private Table createTable(String dbName, String tblName, String owner,
       Map<String,String> tableParams, Map<String, String> partitionKeys,
       StorageDescriptor sd, int lastAccessTime) throws Exception {
@@ -2852,6 +2875,38 @@ public abstract class TestHiveMetaStore extends TestCase {
 
   }
 
+  /**
+   * Test table objects can be retrieved in batches
+   * @throws Exception
+   */
+  @Test
+  public void testGetTableObjects() throws Exception {
+    String dbName = "db";
+    List<String> tableNames = Arrays.asList("table1", "table2", "table3", "table4", "table5");
+
+    // Setup
+    silentDropDatabase(dbName);
+
+    Database db = new Database();
+    db.setName(dbName);
+    client.createDatabase(db);
+    for (String tableName : tableNames) {
+      createTable(dbName, tableName);
+    }
+
+    // Test
+    List<Table> tableObjs = client.getTableObjectsByName(dbName, tableNames);
+
+    // Verify
+    assertEquals(tableNames.size(), tableObjs.size());
+    for(Table table : tableObjs) {
+      assertTrue(tableNames.contains(table.getTableName().toLowerCase()));
+    }
+
+    // Cleanup
+    client.dropDatabase(dbName, true, true, true);
+  }
+
   private void checkDbOwnerType(String dbName, String ownerName, PrincipalType ownerType)
       throws NoSuchObjectException, MetaException, TException {
     Database db = client.getDatabase(dbName);

http://git-wip-us.apache.org/repos/asf/hive/blob/178b8d17/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 0edf11f..ee2cea0 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -26,6 +26,7 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableListMultimap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Multimaps;
+
 import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -186,6 +187,7 @@ import org.apache.hadoop.hive.thrift.TUGIContainingTransport;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hive.common.util.HiveStringUtils;
 import org.apache.thrift.TException;
 import org.apache.thrift.TProcessor;
 import org.apache.thrift.protocol.TBinaryProtocol;
@@ -203,6 +205,7 @@ import org.apache.thrift.transport.TTransport;
 import org.apache.thrift.transport.TTransportFactory;
 
 import javax.jdo.JDOException;
+
 import java.io.IOException;
 import java.text.DateFormat;
 import java.text.SimpleDateFormat;
@@ -1831,9 +1834,9 @@ public class HiveMetaStore extends ThriftHiveMetastore {
     /**
      * Gets multiple tables from the hive metastore.
      *
-     * @param dbname
+     * @param dbName
      *          The name of the database in which the tables reside
-     * @param names
+     * @param tableNames
      *          The names of the tables to get.
      *
      * @return A list of tables whose names are in the the list "names" and
@@ -1845,21 +1848,44 @@ public class HiveMetaStore extends ThriftHiveMetastore {
      * @throws UnknownDBException
      */
     @Override
-    public List<Table> get_table_objects_by_name(final String dbname, final List<String> names)
+    public List<Table> get_table_objects_by_name(final String dbName, final List<String> tableNames)
         throws MetaException, InvalidOperationException, UnknownDBException {
-      List<Table> tables = null;
-      startMultiTableFunction("get_multi_table", dbname, names);
+      List<Table> tables = new ArrayList<Table>();
+      startMultiTableFunction("get_multi_table", dbName, tableNames);
       Exception ex = null;
-      try {
+      int tableBatchSize = HiveConf.getIntVar(hiveConf,
+          ConfVars.METASTORE_BATCH_RETRIEVE_MAX);
 
-        if (dbname == null || dbname.isEmpty()) {
+      try {
+        if (dbName == null || dbName.isEmpty()) {
           throw new UnknownDBException("DB name is null or empty");
         }
-        if (names == null)
+        if (tableNames == null)
         {
-          throw new InvalidOperationException(dbname + " cannot find null tables");
+          throw new InvalidOperationException(dbName + " cannot find null tables");
+        }
+
+        // The list of table names could contain duplicates. RawStore.getTableObjectsByName()
+        // only guarantees returning no duplicate table objects in one batch. If we need
+        // to break into multiple batches, remove duplicates first.
+        List<String> distinctTableNames = tableNames;
+        if (distinctTableNames.size() > tableBatchSize) {
+          List<String> lowercaseTableNames = new ArrayList<String>();
+          for (String tableName : tableNames) {
+            lowercaseTableNames.add(HiveStringUtils.normalizeIdentifier(tableName));
+          }
+          distinctTableNames = new ArrayList<String>(new HashSet<String>(lowercaseTableNames));
+        }
+
+        RawStore ms = getMS();
+        int startIndex = 0;
+        // Retrieve the tables from the metastore in batches. Some databases like
+        // Oracle cannot have over 1000 expressions in a in-list
+        while (startIndex < distinctTableNames.size()) {
+          int endIndex = Math.min(startIndex + tableBatchSize, distinctTableNames.size());
+          tables.addAll(ms.getTableObjectsByName(dbName, distinctTableNames.subList(startIndex, endIndex)));
+          startIndex = endIndex;
         }
-        tables = getMS().getTableObjectsByName(dbname, names);
       } catch (Exception e) {
         ex = e;
         if (e instanceof MetaException) {
@@ -1872,7 +1898,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
           throw newMetaException(e);
         }
       } finally {
-        endFunction("get_multi_table", tables != null, ex, join(names, ","));
+        endFunction("get_multi_table", tables != null, ex, join(tableNames, ","));
       }
       return tables;
     }


[37/50] [abbrv] 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 ga...@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/hbase-metastore
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


[16/50] [abbrv] hive git commit: HIVE-11145 Remove OFFLINE and NO_DROP from tables and partitions (gates, reviewed by Ashutosh Chauhan)

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/describe_syntax.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/describe_syntax.q.out b/ql/src/test/results/clientpositive/describe_syntax.q.out
index cb6f40e..23ca546 100644
--- a/ql/src/test/results/clientpositive/describe_syntax.q.out
+++ b/ql/src/test/results/clientpositive/describe_syntax.q.out
@@ -95,7 +95,6 @@ part                	string
 # Detailed Table Information	 	 
 Database:           	db1                 	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -168,7 +167,6 @@ part                	string
 # Detailed Table Information	 	 
 Database:           	db1                 	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -322,8 +320,6 @@ Partition Value:    	[4, 5]
 Database:           	db1                 	 
 Table:              	t1                  	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 #### A masked pattern was here ####
 	 	 
@@ -395,8 +391,6 @@ Partition Value:    	[4, 5]
 Database:           	db1                 	 
 Table:              	t1                  	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 #### A masked pattern was here ####
 	 	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/describe_table.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/describe_table.q.out b/ql/src/test/results/clientpositive/describe_table.q.out
index 68003cc..fc06a1f 100644
--- a/ql/src/test/results/clientpositive/describe_table.q.out
+++ b/ql/src/test/results/clientpositive/describe_table.q.out
@@ -182,7 +182,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -230,8 +229,6 @@ Partition Value:    	[2008-04-08, 12]
 Database:           	default             	 
 Table:              	srcpart             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -270,7 +267,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -318,8 +314,6 @@ Partition Value:    	[2008-04-08, 12]
 Database:           	default             	 
 Table:              	srcpart             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -398,7 +392,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/dynpart_sort_opt_vectorization.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/dynpart_sort_opt_vectorization.q.out b/ql/src/test/results/clientpositive/dynpart_sort_opt_vectorization.q.out
index 4c8ddd3..216a79c 100644
--- a/ql/src/test/results/clientpositive/dynpart_sort_opt_vectorization.q.out
+++ b/ql/src/test/results/clientpositive/dynpart_sort_opt_vectorization.q.out
@@ -889,8 +889,6 @@ Partition Value:    	[foo, 27]
 Database:           	default             	 
 Table:              	over1k_part_orc     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   
@@ -933,8 +931,6 @@ Partition Value:    	[foo, __HIVE_DEFAULT_PARTITION__]
 Database:           	default             	 
 Table:              	over1k_part_orc     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   
@@ -977,8 +973,6 @@ Partition Value:    	[foo, 27]
 Database:           	default             	 
 Table:              	over1k_part_limit_orc	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   
@@ -1021,8 +1015,6 @@ Partition Value:    	[foo, __HIVE_DEFAULT_PARTITION__]
 Database:           	default             	 
 Table:              	over1k_part_limit_orc	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   
@@ -1064,8 +1056,6 @@ Partition Value:    	[27]
 Database:           	default             	 
 Table:              	over1k_part_buck_orc	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	8                   
@@ -1107,8 +1097,6 @@ Partition Value:    	[__HIVE_DEFAULT_PARTITION__]
 Database:           	default             	 
 Table:              	over1k_part_buck_orc	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	8                   
@@ -1150,8 +1138,6 @@ Partition Value:    	[27]
 Database:           	default             	 
 Table:              	over1k_part_buck_sort_orc	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	8                   
@@ -1193,8 +1179,6 @@ Partition Value:    	[__HIVE_DEFAULT_PARTITION__]
 Database:           	default             	 
 Table:              	over1k_part_buck_sort_orc	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	8                   
@@ -1711,8 +1695,6 @@ Partition Value:    	[foo, 27]
 Database:           	default             	 
 Table:              	over1k_part2_orc    	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1755,8 +1737,6 @@ Partition Value:    	[foo, __HIVE_DEFAULT_PARTITION__]
 Database:           	default             	 
 Table:              	over1k_part2_orc    	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1860,8 +1840,6 @@ Partition Value:    	[foo, 27]
 Database:           	default             	 
 Table:              	over1k_part2_orc    	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1904,8 +1882,6 @@ Partition Value:    	[foo, __HIVE_DEFAULT_PARTITION__]
 Database:           	default             	 
 Table:              	over1k_part2_orc    	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -2154,8 +2130,6 @@ Partition Value:    	[27]
 Database:           	default             	 
 Table:              	over1k_part_buck_sort2_orc	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -2197,8 +2171,6 @@ Partition Value:    	[__HIVE_DEFAULT_PARTITION__]
 Database:           	default             	 
 Table:              	over1k_part_buck_sort2_orc	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -2368,8 +2340,6 @@ Partition Value:    	[27]
 Database:           	default             	 
 Table:              	over1k_part_buck_sort2_orc	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -2411,8 +2381,6 @@ Partition Value:    	[__HIVE_DEFAULT_PARTITION__]
 Database:           	default             	 
 Table:              	over1k_part_buck_sort2_orc	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/dynpart_sort_optimization.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/dynpart_sort_optimization.q.out b/ql/src/test/results/clientpositive/dynpart_sort_optimization.q.out
index 9e947bb..41049bd 100644
--- a/ql/src/test/results/clientpositive/dynpart_sort_optimization.q.out
+++ b/ql/src/test/results/clientpositive/dynpart_sort_optimization.q.out
@@ -794,8 +794,6 @@ Partition Value:    	[foo, 27]
 Database:           	default             	 
 Table:              	over1k_part         	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   
@@ -838,8 +836,6 @@ Partition Value:    	[foo, __HIVE_DEFAULT_PARTITION__]
 Database:           	default             	 
 Table:              	over1k_part         	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   
@@ -882,8 +878,6 @@ Partition Value:    	[foo, 27]
 Database:           	default             	 
 Table:              	over1k_part_limit   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   
@@ -926,8 +920,6 @@ Partition Value:    	[foo, __HIVE_DEFAULT_PARTITION__]
 Database:           	default             	 
 Table:              	over1k_part_limit   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   
@@ -969,8 +961,6 @@ Partition Value:    	[27]
 Database:           	default             	 
 Table:              	over1k_part_buck    	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	8                   
@@ -1012,8 +1002,6 @@ Partition Value:    	[__HIVE_DEFAULT_PARTITION__]
 Database:           	default             	 
 Table:              	over1k_part_buck    	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	8                   
@@ -1055,8 +1043,6 @@ Partition Value:    	[27]
 Database:           	default             	 
 Table:              	over1k_part_buck_sort	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	8                   
@@ -1098,8 +1084,6 @@ Partition Value:    	[__HIVE_DEFAULT_PARTITION__]
 Database:           	default             	 
 Table:              	over1k_part_buck_sort	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	8                   
@@ -1611,8 +1595,6 @@ Partition Value:    	[foo, 27]
 Database:           	default             	 
 Table:              	over1k_part2        	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1655,8 +1637,6 @@ Partition Value:    	[foo, __HIVE_DEFAULT_PARTITION__]
 Database:           	default             	 
 Table:              	over1k_part2        	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1760,8 +1740,6 @@ Partition Value:    	[foo, 27]
 Database:           	default             	 
 Table:              	over1k_part2        	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1804,8 +1782,6 @@ Partition Value:    	[foo, __HIVE_DEFAULT_PARTITION__]
 Database:           	default             	 
 Table:              	over1k_part2        	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -2052,8 +2028,6 @@ Partition Value:    	[27]
 Database:           	default             	 
 Table:              	over1k_part_buck_sort2	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -2095,8 +2069,6 @@ Partition Value:    	[__HIVE_DEFAULT_PARTITION__]
 Database:           	default             	 
 Table:              	over1k_part_buck_sort2	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -2199,8 +2171,6 @@ Partition Value:    	[27]
 Database:           	default             	 
 Table:              	over1k_part_buck_sort2	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -2242,8 +2212,6 @@ Partition Value:    	[__HIVE_DEFAULT_PARTITION__]
 Database:           	default             	 
 Table:              	over1k_part_buck_sort2	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/dynpart_sort_optimization2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/dynpart_sort_optimization2.q.out b/ql/src/test/results/clientpositive/dynpart_sort_optimization2.q.out
index 4dda248..cb0eb58 100644
--- a/ql/src/test/results/clientpositive/dynpart_sort_optimization2.q.out
+++ b/ql/src/test/results/clientpositive/dynpart_sort_optimization2.q.out
@@ -187,8 +187,6 @@ Partition Value:    	[2452617]
 Database:           	default             	 
 Table:              	ss_part             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -249,8 +247,6 @@ Partition Value:    	[2452638]
 Database:           	default             	 
 Table:              	ss_part             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -408,8 +404,6 @@ Partition Value:    	[2452617]
 Database:           	default             	 
 Table:              	ss_part             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -470,8 +464,6 @@ Partition Value:    	[2452638]
 Database:           	default             	 
 Table:              	ss_part             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -650,8 +642,6 @@ Partition Value:    	[2452617]
 Database:           	default             	 
 Table:              	ss_part             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -712,8 +702,6 @@ Partition Value:    	[2452638]
 Database:           	default             	 
 Table:              	ss_part             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -870,8 +858,6 @@ Partition Value:    	[2452617]
 Database:           	default             	 
 Table:              	ss_part             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -932,8 +918,6 @@ Partition Value:    	[2452638]
 Database:           	default             	 
 Table:              	ss_part             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1168,8 +1152,6 @@ Partition Value:    	[2452617]
 Database:           	default             	 
 Table:              	ss_part_orc         	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1230,8 +1212,6 @@ Partition Value:    	[2452638]
 Database:           	default             	 
 Table:              	ss_part_orc         	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1389,8 +1369,6 @@ Partition Value:    	[2452617]
 Database:           	default             	 
 Table:              	ss_part_orc         	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1451,8 +1429,6 @@ Partition Value:    	[2452638]
 Database:           	default             	 
 Table:              	ss_part_orc         	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/encrypted/encryption_insert_values.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/encrypted/encryption_insert_values.q.out b/ql/src/test/results/clientpositive/encrypted/encryption_insert_values.q.out
index 888a612..c1cbf30 100644
--- a/ql/src/test/results/clientpositive/encrypted/encryption_insert_values.q.out
+++ b/ql/src/test/results/clientpositive/encrypted/encryption_insert_values.q.out
@@ -54,7 +54,6 @@ tmp_values_col2     	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A PARTIAL masked pattern was here #### data/warehouse/default/encrypted_table/.hive-staging
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/exim_hidden_files.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/exim_hidden_files.q.out b/ql/src/test/results/clientpositive/exim_hidden_files.q.out
index e449e0e..8076148 100644
--- a/ql/src/test/results/clientpositive/exim_hidden_files.q.out
+++ b/ql/src/test/results/clientpositive/exim_hidden_files.q.out
@@ -69,7 +69,6 @@ emp_country         	string
 # Detailed Table Information	 	 
 Database:           	importer            	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/index_skewtable.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/index_skewtable.q.out b/ql/src/test/results/clientpositive/index_skewtable.q.out
index 348c742..10a441d 100644
--- a/ql/src/test/results/clientpositive/index_skewtable.q.out
+++ b/ql/src/test/results/clientpositive/index_skewtable.q.out
@@ -44,7 +44,6 @@ _offsets            	array<bigint>
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	INDEX_TABLE         	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/infer_bucket_sort.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/infer_bucket_sort.q.out b/ql/src/test/results/clientpositive/infer_bucket_sort.q.out
index 3566fa0..5d69e2f 100644
--- a/ql/src/test/results/clientpositive/infer_bucket_sort.q.out
+++ b/ql/src/test/results/clientpositive/infer_bucket_sort.q.out
@@ -47,8 +47,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -102,8 +100,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -157,8 +153,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -212,8 +206,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -267,8 +259,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -322,8 +312,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -377,8 +365,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -432,8 +418,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -487,8 +471,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -542,8 +524,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -597,8 +577,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -652,8 +630,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -707,8 +683,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -762,8 +736,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -817,8 +789,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -872,8 +842,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -927,8 +895,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -982,8 +948,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1037,8 +1001,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1094,8 +1056,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1151,8 +1111,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1206,8 +1164,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1261,8 +1217,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1316,8 +1270,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1371,8 +1323,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/infer_bucket_sort_bucketed_table.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/infer_bucket_sort_bucketed_table.q.out b/ql/src/test/results/clientpositive/infer_bucket_sort_bucketed_table.q.out
index e0b5866..9503897 100644
--- a/ql/src/test/results/clientpositive/infer_bucket_sort_bucketed_table.q.out
+++ b/ql/src/test/results/clientpositive/infer_bucket_sort_bucketed_table.q.out
@@ -49,8 +49,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table_bucketed 	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	3                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/infer_bucket_sort_convert_join.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/infer_bucket_sort_convert_join.q.out b/ql/src/test/results/clientpositive/infer_bucket_sort_convert_join.q.out
index 69df6d0..d1a6789 100644
--- a/ql/src/test/results/clientpositive/infer_bucket_sort_convert_join.q.out
+++ b/ql/src/test/results/clientpositive/infer_bucket_sort_convert_join.q.out
@@ -49,8 +49,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -110,8 +108,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/infer_bucket_sort_dyn_part.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/infer_bucket_sort_dyn_part.q.out b/ql/src/test/results/clientpositive/infer_bucket_sort_dyn_part.q.out
index 773a2a8..c2f0810 100644
--- a/ql/src/test/results/clientpositive/infer_bucket_sort_dyn_part.q.out
+++ b/ql/src/test/results/clientpositive/infer_bucket_sort_dyn_part.q.out
@@ -69,8 +69,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -111,8 +109,6 @@ Partition Value:    	[2008-04-08, 12]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -182,8 +178,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -224,8 +218,6 @@ Partition Value:    	[2008-04-08, 12]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -295,8 +287,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -337,8 +327,6 @@ Partition Value:    	[2008-04-08, 12]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -611,8 +599,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -653,8 +639,6 @@ Partition Value:    	[2008-04-08, 12]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/infer_bucket_sort_grouping_operators.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/infer_bucket_sort_grouping_operators.q.out b/ql/src/test/results/clientpositive/infer_bucket_sort_grouping_operators.q.out
index 413e7b3..a621be8 100644
--- a/ql/src/test/results/clientpositive/infer_bucket_sort_grouping_operators.q.out
+++ b/ql/src/test/results/clientpositive/infer_bucket_sort_grouping_operators.q.out
@@ -129,8 +129,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table_out      	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -190,8 +188,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table_out_2    	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -317,8 +313,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table_out      	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -378,8 +372,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table_out_2    	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -505,8 +497,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table_out      	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -566,8 +556,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table_out_2    	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/infer_bucket_sort_list_bucket.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/infer_bucket_sort_list_bucket.q.out b/ql/src/test/results/clientpositive/infer_bucket_sort_list_bucket.q.out
index 4879cc4..40c2bf3 100644
--- a/ql/src/test/results/clientpositive/infer_bucket_sort_list_bucket.q.out
+++ b/ql/src/test/results/clientpositive/infer_bucket_sort_list_bucket.q.out
@@ -59,8 +59,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	list_bucketing_table	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   
@@ -135,8 +133,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	list_bucketing_table2	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/infer_bucket_sort_map_operators.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/infer_bucket_sort_map_operators.q.out b/ql/src/test/results/clientpositive/infer_bucket_sort_map_operators.q.out
index 703e52c..6aef463 100644
--- a/ql/src/test/results/clientpositive/infer_bucket_sort_map_operators.q.out
+++ b/ql/src/test/results/clientpositive/infer_bucket_sort_map_operators.q.out
@@ -185,8 +185,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table_out      	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -348,8 +346,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table_out      	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -503,8 +499,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table_out      	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -631,8 +625,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table_out      	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/infer_bucket_sort_merge.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/infer_bucket_sort_merge.q.out b/ql/src/test/results/clientpositive/infer_bucket_sort_merge.q.out
index de62198..4ed6d23 100644
--- a/ql/src/test/results/clientpositive/infer_bucket_sort_merge.q.out
+++ b/ql/src/test/results/clientpositive/infer_bucket_sort_merge.q.out
@@ -49,8 +49,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -104,8 +102,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/infer_bucket_sort_multi_insert.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/infer_bucket_sort_multi_insert.q.out b/ql/src/test/results/clientpositive/infer_bucket_sort_multi_insert.q.out
index c7332ea..523c105 100644
--- a/ql/src/test/results/clientpositive/infer_bucket_sort_multi_insert.q.out
+++ b/ql/src/test/results/clientpositive/infer_bucket_sort_multi_insert.q.out
@@ -57,8 +57,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -98,8 +96,6 @@ Partition Value:    	[2]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -161,8 +157,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -202,8 +196,6 @@ Partition Value:    	[2]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -265,8 +257,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -306,8 +296,6 @@ Partition Value:    	[2]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -369,8 +357,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -410,8 +396,6 @@ Partition Value:    	[2]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/infer_bucket_sort_num_buckets.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/infer_bucket_sort_num_buckets.q.out b/ql/src/test/results/clientpositive/infer_bucket_sort_num_buckets.q.out
index 9a3ec57..ce31548 100644
--- a/ql/src/test/results/clientpositive/infer_bucket_sort_num_buckets.q.out
+++ b/ql/src/test/results/clientpositive/infer_bucket_sort_num_buckets.q.out
@@ -148,8 +148,6 @@ Partition Value:    	[2008-04-08, 0]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   
@@ -190,8 +188,6 @@ Partition Value:    	[2008-04-08, 1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/infer_bucket_sort_reducers_power_two.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/infer_bucket_sort_reducers_power_two.q.out b/ql/src/test/results/clientpositive/infer_bucket_sort_reducers_power_two.q.out
index d5910ee..7b002f5 100644
--- a/ql/src/test/results/clientpositive/infer_bucket_sort_reducers_power_two.q.out
+++ b/ql/src/test/results/clientpositive/infer_bucket_sort_reducers_power_two.q.out
@@ -49,8 +49,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	4                   
@@ -104,8 +102,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	4                   
@@ -159,8 +155,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	4                   
@@ -214,8 +208,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	4                   
@@ -269,8 +261,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	16                  
@@ -326,8 +316,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/lb_fs_stats.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/lb_fs_stats.q.out b/ql/src/test/results/clientpositive/lb_fs_stats.q.out
index 9461714..e09c406 100644
--- a/ql/src/test/results/clientpositive/lb_fs_stats.q.out
+++ b/ql/src/test/results/clientpositive/lb_fs_stats.q.out
@@ -53,8 +53,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_tab            	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/list_bucket_dml_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/list_bucket_dml_1.q.out b/ql/src/test/results/clientpositive/list_bucket_dml_1.q.out
index 7cce327..067ac5d 100644
--- a/ql/src/test/results/clientpositive/list_bucket_dml_1.q.out
+++ b/ql/src/test/results/clientpositive/list_bucket_dml_1.q.out
@@ -282,8 +282,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	list_bucketing_dynamic_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   
@@ -329,8 +327,6 @@ Partition Value:    	[2008-04-08, 12]
 Database:           	default             	 
 Table:              	list_bucketing_dynamic_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/list_bucket_dml_10.q.java1.7.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/list_bucket_dml_10.q.java1.7.out b/ql/src/test/results/clientpositive/list_bucket_dml_10.q.java1.7.out
index cfaadd8..1bd4755 100644
--- a/ql/src/test/results/clientpositive/list_bucket_dml_10.q.java1.7.out
+++ b/ql/src/test/results/clientpositive/list_bucket_dml_10.q.java1.7.out
@@ -364,8 +364,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	list_bucketing_static_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	4                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/list_bucket_dml_10.q.java1.8.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/list_bucket_dml_10.q.java1.8.out b/ql/src/test/results/clientpositive/list_bucket_dml_10.q.java1.8.out
index f872301..d1b9598 100644
--- a/ql/src/test/results/clientpositive/list_bucket_dml_10.q.java1.8.out
+++ b/ql/src/test/results/clientpositive/list_bucket_dml_10.q.java1.8.out
@@ -364,8 +364,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	list_bucketing_static_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	4                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/list_bucket_dml_11.q.java1.7.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/list_bucket_dml_11.q.java1.7.out b/ql/src/test/results/clientpositive/list_bucket_dml_11.q.java1.7.out
index e7b5540..8cc1370 100644
--- a/ql/src/test/results/clientpositive/list_bucket_dml_11.q.java1.7.out
+++ b/ql/src/test/results/clientpositive/list_bucket_dml_11.q.java1.7.out
@@ -238,8 +238,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	list_bucketing_static_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	4                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/list_bucket_dml_11.q.java1.8.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/list_bucket_dml_11.q.java1.8.out b/ql/src/test/results/clientpositive/list_bucket_dml_11.q.java1.8.out
index c7eeb8b..00a6235 100644
--- a/ql/src/test/results/clientpositive/list_bucket_dml_11.q.java1.8.out
+++ b/ql/src/test/results/clientpositive/list_bucket_dml_11.q.java1.8.out
@@ -238,8 +238,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	list_bucketing_static_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	4                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/list_bucket_dml_12.q.java1.7.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/list_bucket_dml_12.q.java1.7.out b/ql/src/test/results/clientpositive/list_bucket_dml_12.q.java1.7.out
index 5b5a35a..b0735a3 100644
--- a/ql/src/test/results/clientpositive/list_bucket_dml_12.q.java1.7.out
+++ b/ql/src/test/results/clientpositive/list_bucket_dml_12.q.java1.7.out
@@ -248,8 +248,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	list_bucketing_mul_col	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	4                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/list_bucket_dml_12.q.java1.8.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/list_bucket_dml_12.q.java1.8.out b/ql/src/test/results/clientpositive/list_bucket_dml_12.q.java1.8.out
index 677cc7d..6d2298b 100644
--- a/ql/src/test/results/clientpositive/list_bucket_dml_12.q.java1.8.out
+++ b/ql/src/test/results/clientpositive/list_bucket_dml_12.q.java1.8.out
@@ -248,8 +248,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	list_bucketing_mul_col	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	4                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/list_bucket_dml_13.q.java1.7.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/list_bucket_dml_13.q.java1.7.out b/ql/src/test/results/clientpositive/list_bucket_dml_13.q.java1.7.out
index dc07f10..6761092 100644
--- a/ql/src/test/results/clientpositive/list_bucket_dml_13.q.java1.7.out
+++ b/ql/src/test/results/clientpositive/list_bucket_dml_13.q.java1.7.out
@@ -248,8 +248,6 @@ Partition Value:    	[2008-04-08, 2013-01-23+18:00:99]
 Database:           	default             	 
 Table:              	list_bucketing_mul_col	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	4                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/list_bucket_dml_13.q.java1.8.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/list_bucket_dml_13.q.java1.8.out b/ql/src/test/results/clientpositive/list_bucket_dml_13.q.java1.8.out
index 71ffd0e..f7a1039 100644
--- a/ql/src/test/results/clientpositive/list_bucket_dml_13.q.java1.8.out
+++ b/ql/src/test/results/clientpositive/list_bucket_dml_13.q.java1.8.out
@@ -248,8 +248,6 @@ Partition Value:    	[2008-04-08, 2013-01-23+18:00:99]
 Database:           	default             	 
 Table:              	list_bucketing_mul_col	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	4                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/list_bucket_dml_14.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/list_bucket_dml_14.q.out b/ql/src/test/results/clientpositive/list_bucket_dml_14.q.out
index 27dcb9e..1519b96 100644
--- a/ql/src/test/results/clientpositive/list_bucket_dml_14.q.out
+++ b/ql/src/test/results/clientpositive/list_bucket_dml_14.q.out
@@ -190,7 +190,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/list_bucket_dml_2.q.java1.7.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/list_bucket_dml_2.q.java1.7.out b/ql/src/test/results/clientpositive/list_bucket_dml_2.q.java1.7.out
index c52c7d8..59bb498 100644
--- a/ql/src/test/results/clientpositive/list_bucket_dml_2.q.java1.7.out
+++ b/ql/src/test/results/clientpositive/list_bucket_dml_2.q.java1.7.out
@@ -312,8 +312,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	list_bucketing_static_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	6                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/list_bucket_dml_2.q.java1.8.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/list_bucket_dml_2.q.java1.8.out b/ql/src/test/results/clientpositive/list_bucket_dml_2.q.java1.8.out
index 685511f..aeeba03 100644
--- a/ql/src/test/results/clientpositive/list_bucket_dml_2.q.java1.8.out
+++ b/ql/src/test/results/clientpositive/list_bucket_dml_2.q.java1.8.out
@@ -312,8 +312,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	list_bucketing_static_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	6                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/list_bucket_dml_3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/list_bucket_dml_3.q.out b/ql/src/test/results/clientpositive/list_bucket_dml_3.q.out
index 0317ad2..fad6cb9 100644
--- a/ql/src/test/results/clientpositive/list_bucket_dml_3.q.out
+++ b/ql/src/test/results/clientpositive/list_bucket_dml_3.q.out
@@ -271,8 +271,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	list_bucketing_static_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	4                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/list_bucket_dml_4.q.java1.7.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/list_bucket_dml_4.q.java1.7.out b/ql/src/test/results/clientpositive/list_bucket_dml_4.q.java1.7.out
index 976fe63..23d6896 100644
--- a/ql/src/test/results/clientpositive/list_bucket_dml_4.q.java1.7.out
+++ b/ql/src/test/results/clientpositive/list_bucket_dml_4.q.java1.7.out
@@ -320,8 +320,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	list_bucketing_static_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	6                   
@@ -723,8 +721,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	list_bucketing_static_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	4                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/list_bucket_dml_4.q.java1.8.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/list_bucket_dml_4.q.java1.8.out b/ql/src/test/results/clientpositive/list_bucket_dml_4.q.java1.8.out
index a79d478..088639b 100644
--- a/ql/src/test/results/clientpositive/list_bucket_dml_4.q.java1.8.out
+++ b/ql/src/test/results/clientpositive/list_bucket_dml_4.q.java1.8.out
@@ -320,8 +320,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	list_bucketing_static_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	6                   
@@ -723,8 +721,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	list_bucketing_static_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	4                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/list_bucket_dml_5.q.java1.7.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/list_bucket_dml_5.q.java1.7.out b/ql/src/test/results/clientpositive/list_bucket_dml_5.q.java1.7.out
index c91fd5f..ce8bb4a 100644
--- a/ql/src/test/results/clientpositive/list_bucket_dml_5.q.java1.7.out
+++ b/ql/src/test/results/clientpositive/list_bucket_dml_5.q.java1.7.out
@@ -286,8 +286,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	list_bucketing_dynamic_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	3                   
@@ -333,8 +331,6 @@ Partition Value:    	[2008-04-08, 12]
 Database:           	default             	 
 Table:              	list_bucketing_dynamic_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	3                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/list_bucket_dml_5.q.java1.8.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/list_bucket_dml_5.q.java1.8.out b/ql/src/test/results/clientpositive/list_bucket_dml_5.q.java1.8.out
index 831b337..1c33382 100644
--- a/ql/src/test/results/clientpositive/list_bucket_dml_5.q.java1.8.out
+++ b/ql/src/test/results/clientpositive/list_bucket_dml_5.q.java1.8.out
@@ -286,8 +286,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	list_bucketing_dynamic_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	3                   
@@ -333,8 +331,6 @@ Partition Value:    	[2008-04-08, 12]
 Database:           	default             	 
 Table:              	list_bucketing_dynamic_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	3                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/list_bucket_dml_6.q.java1.7.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/list_bucket_dml_6.q.java1.7.out b/ql/src/test/results/clientpositive/list_bucket_dml_6.q.java1.7.out
index 883f086..d223234 100644
--- a/ql/src/test/results/clientpositive/list_bucket_dml_6.q.java1.7.out
+++ b/ql/src/test/results/clientpositive/list_bucket_dml_6.q.java1.7.out
@@ -380,8 +380,6 @@ Partition Value:    	[2008-04-08, a1]
 Database:           	default             	 
 Table:              	list_bucketing_dynamic_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   
@@ -425,8 +423,6 @@ Partition Value:    	[2008-04-08, b1]
 Database:           	default             	 
 Table:              	list_bucketing_dynamic_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	6                   
@@ -848,8 +844,6 @@ Partition Value:    	[2008-04-08, a1]
 Database:           	default             	 
 Table:              	list_bucketing_dynamic_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -893,8 +887,6 @@ Partition Value:    	[2008-04-08, b1]
 Database:           	default             	 
 Table:              	list_bucketing_dynamic_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	4                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/list_bucket_dml_6.q.java1.8.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/list_bucket_dml_6.q.java1.8.out b/ql/src/test/results/clientpositive/list_bucket_dml_6.q.java1.8.out
index e968402..f884ace 100644
--- a/ql/src/test/results/clientpositive/list_bucket_dml_6.q.java1.8.out
+++ b/ql/src/test/results/clientpositive/list_bucket_dml_6.q.java1.8.out
@@ -380,8 +380,6 @@ Partition Value:    	[2008-04-08, a1]
 Database:           	default             	 
 Table:              	list_bucketing_dynamic_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   
@@ -425,8 +423,6 @@ Partition Value:    	[2008-04-08, b1]
 Database:           	default             	 
 Table:              	list_bucketing_dynamic_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	6                   
@@ -848,8 +844,6 @@ Partition Value:    	[2008-04-08, a1]
 Database:           	default             	 
 Table:              	list_bucketing_dynamic_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -893,8 +887,6 @@ Partition Value:    	[2008-04-08, b1]
 Database:           	default             	 
 Table:              	list_bucketing_dynamic_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	4                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/list_bucket_dml_7.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/list_bucket_dml_7.q.out b/ql/src/test/results/clientpositive/list_bucket_dml_7.q.out
index 2a8efc4..541944d 100644
--- a/ql/src/test/results/clientpositive/list_bucket_dml_7.q.out
+++ b/ql/src/test/results/clientpositive/list_bucket_dml_7.q.out
@@ -326,8 +326,6 @@ Partition Value:    	[2008-04-08, a1]
 Database:           	default             	 
 Table:              	list_bucketing_dynamic_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   
@@ -371,8 +369,6 @@ Partition Value:    	[2008-04-08, b1]
 Database:           	default             	 
 Table:              	list_bucketing_dynamic_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	4                   
@@ -794,8 +790,6 @@ Partition Value:    	[2008-04-08, a1]
 Database:           	default             	 
 Table:              	list_bucketing_dynamic_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -839,8 +833,6 @@ Partition Value:    	[2008-04-08, b1]
 Database:           	default             	 
 Table:              	list_bucketing_dynamic_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	3                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/list_bucket_dml_8.q.java1.7.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/list_bucket_dml_8.q.java1.7.out b/ql/src/test/results/clientpositive/list_bucket_dml_8.q.java1.7.out
index a9522e0..fc22118 100644
--- a/ql/src/test/results/clientpositive/list_bucket_dml_8.q.java1.7.out
+++ b/ql/src/test/results/clientpositive/list_bucket_dml_8.q.java1.7.out
@@ -382,8 +382,6 @@ Partition Value:    	[2008-04-08, a1]
 Database:           	default             	 
 Table:              	list_bucketing_dynamic_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   
@@ -427,8 +425,6 @@ Partition Value:    	[2008-04-08, b1]
 Database:           	default             	 
 Table:              	list_bucketing_dynamic_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	6                   
@@ -484,8 +480,6 @@ Partition Value:    	[2008-04-08, b1]
 Database:           	default             	 
 Table:              	list_bucketing_dynamic_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	3                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/list_bucket_dml_8.q.java1.8.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/list_bucket_dml_8.q.java1.8.out b/ql/src/test/results/clientpositive/list_bucket_dml_8.q.java1.8.out
index 41979fc..9947c1a 100644
--- a/ql/src/test/results/clientpositive/list_bucket_dml_8.q.java1.8.out
+++ b/ql/src/test/results/clientpositive/list_bucket_dml_8.q.java1.8.out
@@ -382,8 +382,6 @@ Partition Value:    	[2008-04-08, a1]
 Database:           	default             	 
 Table:              	list_bucketing_dynamic_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   
@@ -427,8 +425,6 @@ Partition Value:    	[2008-04-08, b1]
 Database:           	default             	 
 Table:              	list_bucketing_dynamic_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	6                   
@@ -484,8 +480,6 @@ Partition Value:    	[2008-04-08, b1]
 Database:           	default             	 
 Table:              	list_bucketing_dynamic_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	3                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/list_bucket_dml_9.q.java1.7.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/list_bucket_dml_9.q.java1.7.out b/ql/src/test/results/clientpositive/list_bucket_dml_9.q.java1.7.out
index e62d44a..8975ec0 100644
--- a/ql/src/test/results/clientpositive/list_bucket_dml_9.q.java1.7.out
+++ b/ql/src/test/results/clientpositive/list_bucket_dml_9.q.java1.7.out
@@ -320,8 +320,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	list_bucketing_static_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	6                   
@@ -723,8 +721,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	list_bucketing_static_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	4                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/list_bucket_dml_9.q.java1.8.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/list_bucket_dml_9.q.java1.8.out b/ql/src/test/results/clientpositive/list_bucket_dml_9.q.java1.8.out
index 0178e64..a9bf61a 100644
--- a/ql/src/test/results/clientpositive/list_bucket_dml_9.q.java1.8.out
+++ b/ql/src/test/results/clientpositive/list_bucket_dml_9.q.java1.8.out
@@ -320,8 +320,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	list_bucketing_static_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	6                   
@@ -723,8 +721,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	list_bucketing_static_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	4                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/list_bucket_query_multiskew_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/list_bucket_query_multiskew_1.q.out b/ql/src/test/results/clientpositive/list_bucket_query_multiskew_1.q.out
index b6c5b4c..7c4b70c 100644
--- a/ql/src/test/results/clientpositive/list_bucket_query_multiskew_1.q.out
+++ b/ql/src/test/results/clientpositive/list_bucket_query_multiskew_1.q.out
@@ -74,8 +74,6 @@ Partition Value:    	[1, 4]
 Database:           	default             	 
 Table:              	fact_daily          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	3                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/list_bucket_query_multiskew_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/list_bucket_query_multiskew_2.q.out b/ql/src/test/results/clientpositive/list_bucket_query_multiskew_2.q.out
index 6d4e165..7f32108 100644
--- a/ql/src/test/results/clientpositive/list_bucket_query_multiskew_2.q.out
+++ b/ql/src/test/results/clientpositive/list_bucket_query_multiskew_2.q.out
@@ -74,8 +74,6 @@ Partition Value:    	[1, 4]
 Database:           	default             	 
 Table:              	fact_daily          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	3                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/list_bucket_query_multiskew_3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/list_bucket_query_multiskew_3.q.out b/ql/src/test/results/clientpositive/list_bucket_query_multiskew_3.q.out
index 3c0f5e8..753729f 100644
--- a/ql/src/test/results/clientpositive/list_bucket_query_multiskew_3.q.out
+++ b/ql/src/test/results/clientpositive/list_bucket_query_multiskew_3.q.out
@@ -74,8 +74,6 @@ Partition Value:    	[1, 1]
 Database:           	default             	 
 Table:              	fact_daily          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -138,8 +136,6 @@ Partition Value:    	[1, 2]
 Database:           	default             	 
 Table:              	fact_daily          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	3                   
@@ -207,8 +203,6 @@ Partition Value:    	[1, 3]
 Database:           	default             	 
 Table:              	fact_daily          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   


[31/50] [abbrv] hive git commit: HIVE-11285 : ObjectInspector for partition columns in FetchOperator in SMBJoin causes exception (Pengcheng Xiong via Ashutosh Chauhan)

Posted by ga...@apache.org.
HIVE-11285 : ObjectInspector for partition columns in FetchOperator in SMBJoin causes exception (Pengcheng Xiong via Ashutosh Chauhan)


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

Branch: refs/heads/hbase-metastore
Commit: 8646c12f7438a166d32cf8527733fe68d152a831
Parents: b0247cf
Author: Pengcheng Xiong <px...@hortonworks.com>
Authored: Thu Jul 16 02:09:00 2015 +0700
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Fri Jul 17 13:06:10 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hive/ql/exec/FetchOperator.java      |   5 +-
 .../clientpositive/smb_join_partition_key.q     |  35 +++++
 .../clientpositive/smb_join_partition_key.q.out | 128 +++++++++++++++++++
 3 files changed, 166 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/8646c12f/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java
index 258d28e..4c6f7ee 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java
@@ -229,8 +229,9 @@ public class FetchOperator implements Serializable {
     String[] partKeyTypes = pcolTypes.trim().split(":");
     ObjectInspector[] inspectors = new ObjectInspector[partKeys.length];
     for (int i = 0; i < partKeys.length; i++) {
-      inspectors[i] = TypeInfoUtils.getStandardJavaObjectInspectorFromTypeInfo(
-          TypeInfoFactory.getPrimitiveTypeInfo(partKeyTypes[i]));
+      inspectors[i] = PrimitiveObjectInspectorFactory
+          .getPrimitiveWritableObjectInspector(TypeInfoFactory
+              .getPrimitiveTypeInfo(partKeyTypes[i]));
     }
     return ObjectInspectorFactory.getStandardStructObjectInspector(
         Arrays.asList(partKeys), Arrays.asList(inspectors));

http://git-wip-us.apache.org/repos/asf/hive/blob/8646c12f/ql/src/test/queries/clientpositive/smb_join_partition_key.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/smb_join_partition_key.q b/ql/src/test/queries/clientpositive/smb_join_partition_key.q
new file mode 100644
index 0000000..49e2d2f
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/smb_join_partition_key.q
@@ -0,0 +1,35 @@
+SET hive.execution.engine=mr; 
+SET hive.enforce.sortmergebucketmapjoin=false; 
+SET hive.auto.convert.sortmerge.join=true; 
+SET hive.optimize.bucketmapjoin = true; 
+SET hive.optimize.bucketmapjoin.sortedmerge = true; 
+SET hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat;
+SET hive.exec.dynamic.partition.mode=nonstrict;
+
+CREATE TABLE data_table (key INT, value STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY '|'; 
+
+insert into table data_table values(1, 'one');
+insert into table data_table values(2, 'two');
+
+CREATE TABLE smb_table (key INT, value STRING) CLUSTERED BY (key) 
+SORTED BY (key) INTO 1 BUCKETS STORED AS ORC;
+
+CREATE TABLE smb_table_part (key INT, value STRING) PARTITIONED BY (p1 DECIMAL) 
+CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS 
+STORED AS ORC;
+
+INSERT OVERWRITE TABLE smb_table SELECT * FROM data_table; 
+
+INSERT OVERWRITE TABLE smb_table_part PARTITION (p1) SELECT key, value, 100 as p1 FROM data_table;
+
+SELECT s1.key, s2.p1 FROM smb_table s1 INNER JOIN smb_table_part s2 ON s1.key = s2.key ORDER BY s1.key;
+
+drop table smb_table_part;
+
+CREATE TABLE smb_table_part (key INT, value STRING) PARTITIONED BY (p1 double) 
+CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS 
+STORED AS ORC;
+
+INSERT OVERWRITE TABLE smb_table_part PARTITION (p1) SELECT key, value, 100 as p1 FROM data_table;
+
+SELECT s1.key, s2.p1 FROM smb_table s1 INNER JOIN smb_table_part s2 ON s1.key = s2.key ORDER BY s1.key;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/8646c12f/ql/src/test/results/clientpositive/smb_join_partition_key.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/smb_join_partition_key.q.out b/ql/src/test/results/clientpositive/smb_join_partition_key.q.out
new file mode 100644
index 0000000..a4ab8c3
--- /dev/null
+++ b/ql/src/test/results/clientpositive/smb_join_partition_key.q.out
@@ -0,0 +1,128 @@
+PREHOOK: query: CREATE TABLE data_table (key INT, value STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY '|'
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@data_table
+POSTHOOK: query: CREATE TABLE data_table (key INT, value STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY '|'
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@data_table
+PREHOOK: query: insert into table data_table values(1, 'one')
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__1
+PREHOOK: Output: default@data_table
+POSTHOOK: query: insert into table data_table values(1, 'one')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__1
+POSTHOOK: Output: default@data_table
+POSTHOOK: Lineage: data_table.key EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+POSTHOOK: Lineage: data_table.value SIMPLE [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+PREHOOK: query: insert into table data_table values(2, 'two')
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__2
+PREHOOK: Output: default@data_table
+POSTHOOK: query: insert into table data_table values(2, 'two')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__2
+POSTHOOK: Output: default@data_table
+POSTHOOK: Lineage: data_table.key EXPRESSION [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+POSTHOOK: Lineage: data_table.value SIMPLE [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+PREHOOK: query: CREATE TABLE smb_table (key INT, value STRING) CLUSTERED BY (key) 
+SORTED BY (key) INTO 1 BUCKETS STORED AS ORC
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@smb_table
+POSTHOOK: query: CREATE TABLE smb_table (key INT, value STRING) CLUSTERED BY (key) 
+SORTED BY (key) INTO 1 BUCKETS STORED AS ORC
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@smb_table
+PREHOOK: query: CREATE TABLE smb_table_part (key INT, value STRING) PARTITIONED BY (p1 DECIMAL) 
+CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS 
+STORED AS ORC
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@smb_table_part
+POSTHOOK: query: CREATE TABLE smb_table_part (key INT, value STRING) PARTITIONED BY (p1 DECIMAL) 
+CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS 
+STORED AS ORC
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@smb_table_part
+PREHOOK: query: INSERT OVERWRITE TABLE smb_table SELECT * FROM data_table
+PREHOOK: type: QUERY
+PREHOOK: Input: default@data_table
+PREHOOK: Output: default@smb_table
+POSTHOOK: query: INSERT OVERWRITE TABLE smb_table SELECT * FROM data_table
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@data_table
+POSTHOOK: Output: default@smb_table
+POSTHOOK: Lineage: smb_table.key SIMPLE [(data_table)data_table.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: smb_table.value SIMPLE [(data_table)data_table.FieldSchema(name:value, type:string, comment:null), ]
+PREHOOK: query: INSERT OVERWRITE TABLE smb_table_part PARTITION (p1) SELECT key, value, 100 as p1 FROM data_table
+PREHOOK: type: QUERY
+PREHOOK: Input: default@data_table
+PREHOOK: Output: default@smb_table_part
+POSTHOOK: query: INSERT OVERWRITE TABLE smb_table_part PARTITION (p1) SELECT key, value, 100 as p1 FROM data_table
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@data_table
+POSTHOOK: Output: default@smb_table_part@p1=100
+POSTHOOK: Lineage: smb_table_part PARTITION(p1=100).key SIMPLE [(data_table)data_table.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: smb_table_part PARTITION(p1=100).value SIMPLE [(data_table)data_table.FieldSchema(name:value, type:string, comment:null), ]
+PREHOOK: query: SELECT s1.key, s2.p1 FROM smb_table s1 INNER JOIN smb_table_part s2 ON s1.key = s2.key ORDER BY s1.key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@smb_table
+PREHOOK: Input: default@smb_table_part
+PREHOOK: Input: default@smb_table_part@p1=100
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT s1.key, s2.p1 FROM smb_table s1 INNER JOIN smb_table_part s2 ON s1.key = s2.key ORDER BY s1.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@smb_table
+POSTHOOK: Input: default@smb_table_part
+POSTHOOK: Input: default@smb_table_part@p1=100
+#### A masked pattern was here ####
+1	100
+2	100
+PREHOOK: query: drop table smb_table_part
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@smb_table_part
+PREHOOK: Output: default@smb_table_part
+POSTHOOK: query: drop table smb_table_part
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@smb_table_part
+POSTHOOK: Output: default@smb_table_part
+PREHOOK: query: CREATE TABLE smb_table_part (key INT, value STRING) PARTITIONED BY (p1 double) 
+CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS 
+STORED AS ORC
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@smb_table_part
+POSTHOOK: query: CREATE TABLE smb_table_part (key INT, value STRING) PARTITIONED BY (p1 double) 
+CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS 
+STORED AS ORC
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@smb_table_part
+PREHOOK: query: INSERT OVERWRITE TABLE smb_table_part PARTITION (p1) SELECT key, value, 100 as p1 FROM data_table
+PREHOOK: type: QUERY
+PREHOOK: Input: default@data_table
+PREHOOK: Output: default@smb_table_part
+POSTHOOK: query: INSERT OVERWRITE TABLE smb_table_part PARTITION (p1) SELECT key, value, 100 as p1 FROM data_table
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@data_table
+POSTHOOK: Output: default@smb_table_part@p1=100
+POSTHOOK: Lineage: smb_table_part PARTITION(p1=100).key SIMPLE [(data_table)data_table.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: smb_table_part PARTITION(p1=100).value SIMPLE [(data_table)data_table.FieldSchema(name:value, type:string, comment:null), ]
+PREHOOK: query: SELECT s1.key, s2.p1 FROM smb_table s1 INNER JOIN smb_table_part s2 ON s1.key = s2.key ORDER BY s1.key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@smb_table
+PREHOOK: Input: default@smb_table_part
+PREHOOK: Input: default@smb_table_part@p1=100
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT s1.key, s2.p1 FROM smb_table s1 INNER JOIN smb_table_part s2 ON s1.key = s2.key ORDER BY s1.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@smb_table
+POSTHOOK: Input: default@smb_table_part
+POSTHOOK: Input: default@smb_table_part@p1=100
+#### A masked pattern was here ####
+1	100.0
+2	100.0


[38/50] [abbrv] 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 ga...@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/hbase-metastore
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);


[14/50] [abbrv] hive git commit: HIVE-11145 Remove OFFLINE and NO_DROP from tables and partitions (gates, reviewed by Ashutosh Chauhan)

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/stats_counter_partitioned.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/stats_counter_partitioned.q.out b/ql/src/test/results/clientpositive/spark/stats_counter_partitioned.q.out
index 4b84eca..12e1fbe 100644
--- a/ql/src/test/results/clientpositive/spark/stats_counter_partitioned.q.out
+++ b/ql/src/test/results/clientpositive/spark/stats_counter_partitioned.q.out
@@ -66,8 +66,6 @@ Partition Value:    	[2008, 11]
 Database:           	default             	 
 Table:              	dummy               	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -108,8 +106,6 @@ Partition Value:    	[2008, 12]
 Database:           	default             	 
 Table:              	dummy               	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -190,8 +186,6 @@ Partition Value:    	[10, 11]
 Database:           	default             	 
 Table:              	dummy               	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -232,8 +226,6 @@ Partition Value:    	[10, 12]
 Database:           	default             	 
 Table:              	dummy               	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -323,8 +315,6 @@ Partition Value:    	[1997]
 Database:           	default             	 
 Table:              	dummy               	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -363,8 +353,6 @@ Partition Value:    	[1994]
 Database:           	default             	 
 Table:              	dummy               	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -403,8 +391,6 @@ Partition Value:    	[1998]
 Database:           	default             	 
 Table:              	dummy               	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -443,8 +429,6 @@ Partition Value:    	[1996]
 Database:           	default             	 
 Table:              	dummy               	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/stats_noscan_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/stats_noscan_1.q.out b/ql/src/test/results/clientpositive/spark/stats_noscan_1.q.out
index 80c3092..2559492 100644
--- a/ql/src/test/results/clientpositive/spark/stats_noscan_1.q.out
+++ b/ql/src/test/results/clientpositive/spark/stats_noscan_1.q.out
@@ -100,8 +100,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -142,8 +140,6 @@ Partition Value:    	[2008-04-08, 12]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -184,8 +180,6 @@ Partition Value:    	[2008-04-09, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   
@@ -226,8 +220,6 @@ Partition Value:    	[2008-04-09, 12]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   
@@ -266,7 +258,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -381,8 +372,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart_partial	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -423,8 +412,6 @@ Partition Value:    	[2008-04-08, 12]
 Database:           	default             	 
 Table:              	analyze_srcpart_partial	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -465,8 +452,6 @@ Partition Value:    	[2008-04-09, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart_partial	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   
@@ -507,8 +492,6 @@ Partition Value:    	[2008-04-09, 12]
 Database:           	default             	 
 Table:              	analyze_srcpart_partial	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/stats_noscan_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/stats_noscan_2.q.out b/ql/src/test/results/clientpositive/spark/stats_noscan_2.q.out
index 105c222..8136c39 100644
--- a/ql/src/test/results/clientpositive/spark/stats_noscan_2.q.out
+++ b/ql/src/test/results/clientpositive/spark/stats_noscan_2.q.out
@@ -47,7 +47,6 @@ a                   	int
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	EXTERNAL_TABLE      	 
@@ -91,7 +90,6 @@ a                   	int
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	EXTERNAL_TABLE      	 
@@ -231,8 +229,6 @@ Partition Value:    	[2008-01-01]
 Database:           	default             	 
 Table:              	anaylyze_external   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -284,8 +280,6 @@ Partition Value:    	[2008-01-01]
 Database:           	default             	 
 Table:              	anaylyze_external   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/stats_only_null.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/stats_only_null.q.out b/ql/src/test/results/clientpositive/spark/stats_only_null.q.out
index cceceef..cb0920e 100644
--- a/ql/src/test/results/clientpositive/spark/stats_only_null.q.out
+++ b/ql/src/test/results/clientpositive/spark/stats_only_null.q.out
@@ -229,8 +229,6 @@ Partition Value:    	[2010]
 Database:           	default             	 
 Table:              	stats_null_part     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -272,8 +270,6 @@ Partition Value:    	[2011]
 Database:           	default             	 
 Table:              	stats_null_part     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/stats_partscan_1_23.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/stats_partscan_1_23.q.out b/ql/src/test/results/clientpositive/spark/stats_partscan_1_23.q.out
index adcf150..eb0145b 100644
--- a/ql/src/test/results/clientpositive/spark/stats_partscan_1_23.q.out
+++ b/ql/src/test/results/clientpositive/spark/stats_partscan_1_23.q.out
@@ -75,8 +75,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart_partial_scan	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   
@@ -150,8 +148,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart_partial_scan	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   
@@ -192,8 +188,6 @@ Partition Value:    	[2008-04-09, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart_partial_scan	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/statsfs.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/statsfs.q.out b/ql/src/test/results/clientpositive/spark/statsfs.q.out
index b0bca41..2735f5f 100644
--- a/ql/src/test/results/clientpositive/spark/statsfs.q.out
+++ b/ql/src/test/results/clientpositive/spark/statsfs.q.out
@@ -65,8 +65,6 @@ Partition Value:    	[2010]
 Database:           	default             	 
 Table:              	t1                  	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -106,8 +104,6 @@ Partition Value:    	[2011]
 Database:           	default             	 
 Table:              	t1                  	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -187,8 +183,6 @@ Partition Value:    	[2010]
 Database:           	default             	 
 Table:              	t1                  	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -228,8 +222,6 @@ Partition Value:    	[2011]
 Database:           	default             	 
 Table:              	t1                  	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -298,7 +290,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -364,7 +355,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -457,8 +447,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	t1                  	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -499,8 +487,6 @@ Partition Value:    	[2008-04-09, 12]
 Database:           	default             	 
 Table:              	t1                  	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/union_remove_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union_remove_1.q.out b/ql/src/test/results/clientpositive/spark/union_remove_1.q.out
index c7616b4..a4b5836 100644
--- a/ql/src/test/results/clientpositive/spark/union_remove_1.q.out
+++ b/ql/src/test/results/clientpositive/spark/union_remove_1.q.out
@@ -193,7 +193,6 @@ values              	bigint
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/union_remove_10.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union_remove_10.q.out b/ql/src/test/results/clientpositive/spark/union_remove_10.q.out
index 586b88b..d98b388 100644
--- a/ql/src/test/results/clientpositive/spark/union_remove_10.q.out
+++ b/ql/src/test/results/clientpositive/spark/union_remove_10.q.out
@@ -255,7 +255,6 @@ values              	bigint
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/union_remove_11.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union_remove_11.q.out b/ql/src/test/results/clientpositive/spark/union_remove_11.q.out
index 9ddf606..1e7d5cf 100644
--- a/ql/src/test/results/clientpositive/spark/union_remove_11.q.out
+++ b/ql/src/test/results/clientpositive/spark/union_remove_11.q.out
@@ -245,7 +245,6 @@ values              	bigint
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/union_remove_12.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union_remove_12.q.out b/ql/src/test/results/clientpositive/spark/union_remove_12.q.out
index fdaa941..e57626f 100644
--- a/ql/src/test/results/clientpositive/spark/union_remove_12.q.out
+++ b/ql/src/test/results/clientpositive/spark/union_remove_12.q.out
@@ -245,7 +245,6 @@ values              	bigint
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/union_remove_13.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union_remove_13.q.out b/ql/src/test/results/clientpositive/spark/union_remove_13.q.out
index 995d180..bb294cd 100644
--- a/ql/src/test/results/clientpositive/spark/union_remove_13.q.out
+++ b/ql/src/test/results/clientpositive/spark/union_remove_13.q.out
@@ -271,7 +271,6 @@ values              	bigint
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/union_remove_14.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union_remove_14.q.out b/ql/src/test/results/clientpositive/spark/union_remove_14.q.out
index e23ecf4..3bb9194 100644
--- a/ql/src/test/results/clientpositive/spark/union_remove_14.q.out
+++ b/ql/src/test/results/clientpositive/spark/union_remove_14.q.out
@@ -247,7 +247,6 @@ values              	bigint
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/union_remove_15.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union_remove_15.q.out b/ql/src/test/results/clientpositive/spark/union_remove_15.q.out
index 62dd62c..354a48f 100644
--- a/ql/src/test/results/clientpositive/spark/union_remove_15.q.out
+++ b/ql/src/test/results/clientpositive/spark/union_remove_15.q.out
@@ -217,7 +217,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/union_remove_16.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union_remove_16.q.out b/ql/src/test/results/clientpositive/spark/union_remove_16.q.out
index 42aa20a..57059dc 100644
--- a/ql/src/test/results/clientpositive/spark/union_remove_16.q.out
+++ b/ql/src/test/results/clientpositive/spark/union_remove_16.q.out
@@ -257,7 +257,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/union_remove_17.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union_remove_17.q.out b/ql/src/test/results/clientpositive/spark/union_remove_17.q.out
index c6fdfa5..c8047a9 100644
--- a/ql/src/test/results/clientpositive/spark/union_remove_17.q.out
+++ b/ql/src/test/results/clientpositive/spark/union_remove_17.q.out
@@ -168,7 +168,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/union_remove_18.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union_remove_18.q.out b/ql/src/test/results/clientpositive/spark/union_remove_18.q.out
index d732571..a4d1793 100644
--- a/ql/src/test/results/clientpositive/spark/union_remove_18.q.out
+++ b/ql/src/test/results/clientpositive/spark/union_remove_18.q.out
@@ -227,7 +227,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/union_remove_19.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union_remove_19.q.out b/ql/src/test/results/clientpositive/spark/union_remove_19.q.out
index 982dd67..c6e2070 100644
--- a/ql/src/test/results/clientpositive/spark/union_remove_19.q.out
+++ b/ql/src/test/results/clientpositive/spark/union_remove_19.q.out
@@ -197,7 +197,6 @@ values              	bigint
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/union_remove_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union_remove_2.q.out b/ql/src/test/results/clientpositive/spark/union_remove_2.q.out
index 3993ac5..d886433 100644
--- a/ql/src/test/results/clientpositive/spark/union_remove_2.q.out
+++ b/ql/src/test/results/clientpositive/spark/union_remove_2.q.out
@@ -199,7 +199,6 @@ values              	bigint
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/union_remove_20.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union_remove_20.q.out b/ql/src/test/results/clientpositive/spark/union_remove_20.q.out
index 13d3b5c..4404b87 100644
--- a/ql/src/test/results/clientpositive/spark/union_remove_20.q.out
+++ b/ql/src/test/results/clientpositive/spark/union_remove_20.q.out
@@ -203,7 +203,6 @@ key                 	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/union_remove_21.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union_remove_21.q.out b/ql/src/test/results/clientpositive/spark/union_remove_21.q.out
index f5c7636..ccefacf 100644
--- a/ql/src/test/results/clientpositive/spark/union_remove_21.q.out
+++ b/ql/src/test/results/clientpositive/spark/union_remove_21.q.out
@@ -187,7 +187,6 @@ key                 	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/union_remove_22.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union_remove_22.q.out b/ql/src/test/results/clientpositive/spark/union_remove_22.q.out
index d00fb89..e53e963 100644
--- a/ql/src/test/results/clientpositive/spark/union_remove_22.q.out
+++ b/ql/src/test/results/clientpositive/spark/union_remove_22.q.out
@@ -207,7 +207,6 @@ values2             	bigint
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/union_remove_23.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union_remove_23.q.out b/ql/src/test/results/clientpositive/spark/union_remove_23.q.out
index 3fa6ae0..adb0c62 100644
--- a/ql/src/test/results/clientpositive/spark/union_remove_23.q.out
+++ b/ql/src/test/results/clientpositive/spark/union_remove_23.q.out
@@ -227,7 +227,6 @@ values              	bigint
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/union_remove_24.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union_remove_24.q.out b/ql/src/test/results/clientpositive/spark/union_remove_24.q.out
index c0ce83f..9237316 100644
--- a/ql/src/test/results/clientpositive/spark/union_remove_24.q.out
+++ b/ql/src/test/results/clientpositive/spark/union_remove_24.q.out
@@ -199,7 +199,6 @@ values              	bigint
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/union_remove_25.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union_remove_25.q.out b/ql/src/test/results/clientpositive/spark/union_remove_25.q.out
index 8cd8c8d..6e96186 100644
--- a/ql/src/test/results/clientpositive/spark/union_remove_25.q.out
+++ b/ql/src/test/results/clientpositive/spark/union_remove_25.q.out
@@ -218,8 +218,6 @@ Partition Value:    	[2004]
 Database:           	default             	 
 Table:              	outputtbl1          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	4                   
@@ -429,8 +427,6 @@ Partition Value:    	[2008-04-08]
 Database:           	default             	 
 Table:              	outputtbl2          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	2                   
@@ -624,8 +620,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	outputtbl3          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	2                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/union_remove_3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union_remove_3.q.out b/ql/src/test/results/clientpositive/spark/union_remove_3.q.out
index 2126c1d..565b834 100644
--- a/ql/src/test/results/clientpositive/spark/union_remove_3.q.out
+++ b/ql/src/test/results/clientpositive/spark/union_remove_3.q.out
@@ -189,7 +189,6 @@ values              	bigint
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/union_remove_4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union_remove_4.q.out b/ql/src/test/results/clientpositive/spark/union_remove_4.q.out
index 7dd8e3e..a5730c1 100644
--- a/ql/src/test/results/clientpositive/spark/union_remove_4.q.out
+++ b/ql/src/test/results/clientpositive/spark/union_remove_4.q.out
@@ -243,7 +243,6 @@ values              	bigint
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/union_remove_5.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union_remove_5.q.out b/ql/src/test/results/clientpositive/spark/union_remove_5.q.out
index fe1fd8b..908298d 100644
--- a/ql/src/test/results/clientpositive/spark/union_remove_5.q.out
+++ b/ql/src/test/results/clientpositive/spark/union_remove_5.q.out
@@ -251,7 +251,6 @@ values              	bigint
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/union_remove_7.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union_remove_7.q.out b/ql/src/test/results/clientpositive/spark/union_remove_7.q.out
index 530be3f..5e88d10 100644
--- a/ql/src/test/results/clientpositive/spark/union_remove_7.q.out
+++ b/ql/src/test/results/clientpositive/spark/union_remove_7.q.out
@@ -197,7 +197,6 @@ values              	bigint
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/union_remove_8.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union_remove_8.q.out b/ql/src/test/results/clientpositive/spark/union_remove_8.q.out
index 06adb05..b33767c 100644
--- a/ql/src/test/results/clientpositive/spark/union_remove_8.q.out
+++ b/ql/src/test/results/clientpositive/spark/union_remove_8.q.out
@@ -203,7 +203,6 @@ values              	bigint
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/union_remove_9.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union_remove_9.q.out b/ql/src/test/results/clientpositive/spark/union_remove_9.q.out
index 7857999..e837bd7 100644
--- a/ql/src/test/results/clientpositive/spark/union_remove_9.q.out
+++ b/ql/src/test/results/clientpositive/spark/union_remove_9.q.out
@@ -259,7 +259,6 @@ values              	bigint
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/stats1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/stats1.q.out b/ql/src/test/results/clientpositive/stats1.q.out
index 6f560d3..ac34bbb 100644
--- a/ql/src/test/results/clientpositive/stats1.q.out
+++ b/ql/src/test/results/clientpositive/stats1.q.out
@@ -178,7 +178,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -228,7 +227,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/stats10.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/stats10.q.out b/ql/src/test/results/clientpositive/stats10.q.out
index dc8aa7f..7824cbd 100644
--- a/ql/src/test/results/clientpositive/stats10.q.out
+++ b/ql/src/test/results/clientpositive/stats10.q.out
@@ -413,8 +413,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	bucket3_1           	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   
@@ -454,8 +452,6 @@ Partition Value:    	[2]
 Database:           	default             	 
 Table:              	bucket3_1           	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   
@@ -493,7 +489,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/stats11.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/stats11.q.out b/ql/src/test/results/clientpositive/stats11.q.out
index e51f049..4ed235f 100644
--- a/ql/src/test/results/clientpositive/stats11.q.out
+++ b/ql/src/test/results/clientpositive/stats11.q.out
@@ -86,8 +86,6 @@ Partition Value:    	[2008-04-08]
 Database:           	default             	 
 Table:              	srcbucket_mapjoin_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -135,8 +133,6 @@ Partition Value:    	[2008-04-08]
 Database:           	default             	 
 Table:              	srcbucket_mapjoin_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   
@@ -184,8 +180,6 @@ Partition Value:    	[2008-04-08]
 Database:           	default             	 
 Table:              	srcbucket_mapjoin_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	3                   
@@ -233,8 +227,6 @@ Partition Value:    	[2008-04-08]
 Database:           	default             	 
 Table:              	srcbucket_mapjoin_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	4                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/stats12.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/stats12.q.out b/ql/src/test/results/clientpositive/stats12.q.out
index d5ca59b..c6e7c68 100644
--- a/ql/src/test/results/clientpositive/stats12.q.out
+++ b/ql/src/test/results/clientpositive/stats12.q.out
@@ -205,7 +205,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -244,8 +243,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -286,8 +283,6 @@ Partition Value:    	[2008-04-08, 12]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -328,8 +323,6 @@ Partition Value:    	[2008-04-09, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   
@@ -370,8 +363,6 @@ Partition Value:    	[2008-04-09, 12]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/stats13.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/stats13.q.out b/ql/src/test/results/clientpositive/stats13.q.out
index e34f0b7..7415728 100644
--- a/ql/src/test/results/clientpositive/stats13.q.out
+++ b/ql/src/test/results/clientpositive/stats13.q.out
@@ -155,7 +155,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -194,8 +193,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -236,8 +233,6 @@ Partition Value:    	[2008-04-08, 12]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   
@@ -278,8 +273,6 @@ Partition Value:    	[2008-04-09, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   
@@ -320,8 +313,6 @@ Partition Value:    	[2008-04-09, 12]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   
@@ -368,7 +359,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/stats14.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/stats14.q.out b/ql/src/test/results/clientpositive/stats14.q.out
index f12b136..f34720d 100644
--- a/ql/src/test/results/clientpositive/stats14.q.out
+++ b/ql/src/test/results/clientpositive/stats14.q.out
@@ -38,7 +38,6 @@ value               	string              	default
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -142,7 +141,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -181,8 +179,6 @@ Partition Value:    	[2010-04-08, 11]
 Database:           	default             	 
 Table:              	stats_part          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -223,8 +219,6 @@ Partition Value:    	[2010-04-08, 12]
 Database:           	default             	 
 Table:              	stats_part          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -283,7 +277,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/stats15.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/stats15.q.out b/ql/src/test/results/clientpositive/stats15.q.out
index a60dee2..aad2e3a 100644
--- a/ql/src/test/results/clientpositive/stats15.q.out
+++ b/ql/src/test/results/clientpositive/stats15.q.out
@@ -38,7 +38,6 @@ value               	string              	default
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -142,7 +141,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -181,8 +179,6 @@ Partition Value:    	[2010-04-08, 11]
 Database:           	default             	 
 Table:              	stats_part          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -223,8 +219,6 @@ Partition Value:    	[2010-04-08, 12]
 Database:           	default             	 
 Table:              	stats_part          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -283,7 +277,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/stats16.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/stats16.q.out b/ql/src/test/results/clientpositive/stats16.q.out
index 3f0f2ea..2e3cadb 100644
--- a/ql/src/test/results/clientpositive/stats16.q.out
+++ b/ql/src/test/results/clientpositive/stats16.q.out
@@ -24,7 +24,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -73,7 +72,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/stats18.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/stats18.q.out b/ql/src/test/results/clientpositive/stats18.q.out
index a061846..a7d6ab8 100644
--- a/ql/src/test/results/clientpositive/stats18.q.out
+++ b/ql/src/test/results/clientpositive/stats18.q.out
@@ -44,8 +44,6 @@ Partition Value:    	[2010-04-08, 13]
 Database:           	default             	 
 Table:              	stats_part          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -94,8 +92,6 @@ Partition Value:    	[2010-04-08, 13]
 Database:           	default             	 
 Table:              	stats_part          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/stats19.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/stats19.q.out b/ql/src/test/results/clientpositive/stats19.q.out
index 499a8bd..ea56f3a 100644
--- a/ql/src/test/results/clientpositive/stats19.q.out
+++ b/ql/src/test/results/clientpositive/stats19.q.out
@@ -98,8 +98,6 @@ Partition Value:    	[2010-04-08, 13]
 Database:           	default             	 
 Table:              	stats_part          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -152,8 +150,6 @@ Partition Value:    	[2010-04-08, 13]
 Database:           	default             	 
 Table:              	stats_part          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -206,8 +202,6 @@ Partition Value:    	[2010-04-08, 13]
 Database:           	default             	 
 Table:              	stats_part          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -312,8 +306,6 @@ Partition Value:    	[2010-04-08, 13]
 Database:           	default             	 
 Table:              	stats_part          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -366,8 +358,6 @@ Partition Value:    	[2010-04-08, 13]
 Database:           	default             	 
 Table:              	stats_part          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -420,8 +410,6 @@ Partition Value:    	[2010-04-08, 13]
 Database:           	default             	 
 Table:              	stats_part          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/stats2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/stats2.q.out b/ql/src/test/results/clientpositive/stats2.q.out
index 5e305d3..ac1d5cb 100644
--- a/ql/src/test/results/clientpositive/stats2.q.out
+++ b/ql/src/test/results/clientpositive/stats2.q.out
@@ -96,7 +96,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -176,7 +175,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/stats20.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/stats20.q.out b/ql/src/test/results/clientpositive/stats20.q.out
index 4ac7bc5..d7e52b4 100644
--- a/ql/src/test/results/clientpositive/stats20.q.out
+++ b/ql/src/test/results/clientpositive/stats20.q.out
@@ -39,7 +39,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -89,7 +88,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/stats3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/stats3.q.out b/ql/src/test/results/clientpositive/stats3.q.out
index dd3a95b..2afb76e 100644
--- a/ql/src/test/results/clientpositive/stats3.q.out
+++ b/ql/src/test/results/clientpositive/stats3.q.out
@@ -82,7 +82,6 @@ col1                	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -227,7 +226,6 @@ pcol2               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/stats4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/stats4.q.out b/ql/src/test/results/clientpositive/stats4.q.out
index 39d5413..9ced932 100644
--- a/ql/src/test/results/clientpositive/stats4.q.out
+++ b/ql/src/test/results/clientpositive/stats4.q.out
@@ -2308,8 +2308,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	nzhang_part1        	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -2350,8 +2348,6 @@ Partition Value:    	[2008-04-08, 12]
 Database:           	default             	 
 Table:              	nzhang_part1        	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -2392,8 +2388,6 @@ Partition Value:    	[2008-12-31, 11]
 Database:           	default             	 
 Table:              	nzhang_part2        	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -2434,8 +2428,6 @@ Partition Value:    	[2008-12-31, 12]
 Database:           	default             	 
 Table:              	nzhang_part2        	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -2474,7 +2466,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -2511,7 +2502,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/stats5.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/stats5.q.out b/ql/src/test/results/clientpositive/stats5.q.out
index 484e834..23d4e6b 100644
--- a/ql/src/test/results/clientpositive/stats5.q.out
+++ b/ql/src/test/results/clientpositive/stats5.q.out
@@ -49,7 +49,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/stats6.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/stats6.q.out b/ql/src/test/results/clientpositive/stats6.q.out
index b4435f2..a387075 100644
--- a/ql/src/test/results/clientpositive/stats6.q.out
+++ b/ql/src/test/results/clientpositive/stats6.q.out
@@ -79,8 +79,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -121,8 +119,6 @@ Partition Value:    	[2008-04-08, 12]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -163,8 +159,6 @@ Partition Value:    	[2008-04-09, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   
@@ -205,8 +199,6 @@ Partition Value:    	[2008-04-09, 12]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   
@@ -245,7 +237,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/stats7.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/stats7.q.out b/ql/src/test/results/clientpositive/stats7.q.out
index 03b76e2..7f32764 100644
--- a/ql/src/test/results/clientpositive/stats7.q.out
+++ b/ql/src/test/results/clientpositive/stats7.q.out
@@ -90,8 +90,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -132,8 +130,6 @@ Partition Value:    	[2008-04-08, 12]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -172,7 +168,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/stats8.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/stats8.q.out b/ql/src/test/results/clientpositive/stats8.q.out
index ae632e1..80dd4e8 100644
--- a/ql/src/test/results/clientpositive/stats8.q.out
+++ b/ql/src/test/results/clientpositive/stats8.q.out
@@ -86,8 +86,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -126,7 +124,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -196,8 +193,6 @@ Partition Value:    	[2008-04-08, 12]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -269,8 +264,6 @@ Partition Value:    	[2008-04-09, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -342,8 +335,6 @@ Partition Value:    	[2008-04-09, 12]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -427,8 +418,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -469,8 +458,6 @@ Partition Value:    	[2008-04-08, 12]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -511,8 +498,6 @@ Partition Value:    	[2008-04-09, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -553,8 +538,6 @@ Partition Value:    	[2008-04-09, 12]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -593,7 +576,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/stats9.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/stats9.q.out b/ql/src/test/results/clientpositive/stats9.q.out
index 8563f3b..e7c7743 100644
--- a/ql/src/test/results/clientpositive/stats9.q.out
+++ b/ql/src/test/results/clientpositive/stats9.q.out
@@ -57,7 +57,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/stats_counter.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/stats_counter.q.out b/ql/src/test/results/clientpositive/stats_counter.q.out
index e2980e8..8b3dcea 100644
--- a/ql/src/test/results/clientpositive/stats_counter.q.out
+++ b/ql/src/test/results/clientpositive/stats_counter.q.out
@@ -32,7 +32,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -80,7 +79,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/stats_counter_partitioned.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/stats_counter_partitioned.q.out b/ql/src/test/results/clientpositive/stats_counter_partitioned.q.out
index ab1270c..626dcff 100644
--- a/ql/src/test/results/clientpositive/stats_counter_partitioned.q.out
+++ b/ql/src/test/results/clientpositive/stats_counter_partitioned.q.out
@@ -66,8 +66,6 @@ Partition Value:    	[2008, 11]
 Database:           	default             	 
 Table:              	dummy               	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -108,8 +106,6 @@ Partition Value:    	[2008, 12]
 Database:           	default             	 
 Table:              	dummy               	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -190,8 +186,6 @@ Partition Value:    	[10, 11]
 Database:           	default             	 
 Table:              	dummy               	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -232,8 +226,6 @@ Partition Value:    	[10, 12]
 Database:           	default             	 
 Table:              	dummy               	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -323,8 +315,6 @@ Partition Value:    	[1997]
 Database:           	default             	 
 Table:              	dummy               	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -363,8 +353,6 @@ Partition Value:    	[1994]
 Database:           	default             	 
 Table:              	dummy               	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -403,8 +391,6 @@ Partition Value:    	[1998]
 Database:           	default             	 
 Table:              	dummy               	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -443,8 +429,6 @@ Partition Value:    	[1996]
 Database:           	default             	 
 Table:              	dummy               	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/stats_empty_partition.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/stats_empty_partition.q.out b/ql/src/test/results/clientpositive/stats_empty_partition.q.out
index 65e0a6f..c13817e 100644
--- a/ql/src/test/results/clientpositive/stats_empty_partition.q.out
+++ b/ql/src/test/results/clientpositive/stats_empty_partition.q.out
@@ -43,8 +43,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	tmptable            	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/stats_invalidation.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/stats_invalidation.q.out b/ql/src/test/results/clientpositive/stats_invalidation.q.out
index f8c1731..1bb7dc6 100644
--- a/ql/src/test/results/clientpositive/stats_invalidation.q.out
+++ b/ql/src/test/results/clientpositive/stats_invalidation.q.out
@@ -40,7 +40,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -85,7 +84,6 @@ new_col             	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/stats_list_bucket.q.java1.7.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/stats_list_bucket.q.java1.7.out b/ql/src/test/results/clientpositive/stats_list_bucket.q.java1.7.out
index 5a05eb4..63372c5 100644
--- a/ql/src/test/results/clientpositive/stats_list_bucket.q.java1.7.out
+++ b/ql/src/test/results/clientpositive/stats_list_bucket.q.java1.7.out
@@ -74,8 +74,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	stats_list_bucket   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	4                   
@@ -151,7 +149,6 @@ c2                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/stats_list_bucket.q.java1.8.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/stats_list_bucket.q.java1.8.out b/ql/src/test/results/clientpositive/stats_list_bucket.q.java1.8.out
index 1b26365..8688cee 100644
--- a/ql/src/test/results/clientpositive/stats_list_bucket.q.java1.8.out
+++ b/ql/src/test/results/clientpositive/stats_list_bucket.q.java1.8.out
@@ -74,8 +74,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	stats_list_bucket   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	4                   
@@ -151,7 +149,6 @@ c2                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/stats_noscan_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/stats_noscan_1.q.out b/ql/src/test/results/clientpositive/stats_noscan_1.q.out
index 80c3092..2559492 100644
--- a/ql/src/test/results/clientpositive/stats_noscan_1.q.out
+++ b/ql/src/test/results/clientpositive/stats_noscan_1.q.out
@@ -100,8 +100,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -142,8 +140,6 @@ Partition Value:    	[2008-04-08, 12]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -184,8 +180,6 @@ Partition Value:    	[2008-04-09, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   
@@ -226,8 +220,6 @@ Partition Value:    	[2008-04-09, 12]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   
@@ -266,7 +258,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -381,8 +372,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart_partial	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -423,8 +412,6 @@ Partition Value:    	[2008-04-08, 12]
 Database:           	default             	 
 Table:              	analyze_srcpart_partial	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -465,8 +452,6 @@ Partition Value:    	[2008-04-09, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart_partial	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   
@@ -507,8 +492,6 @@ Partition Value:    	[2008-04-09, 12]
 Database:           	default             	 
 Table:              	analyze_srcpart_partial	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/stats_noscan_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/stats_noscan_2.q.out b/ql/src/test/results/clientpositive/stats_noscan_2.q.out
index 105c222..8136c39 100644
--- a/ql/src/test/results/clientpositive/stats_noscan_2.q.out
+++ b/ql/src/test/results/clientpositive/stats_noscan_2.q.out
@@ -47,7 +47,6 @@ a                   	int
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	EXTERNAL_TABLE      	 
@@ -91,7 +90,6 @@ a                   	int
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	EXTERNAL_TABLE      	 
@@ -231,8 +229,6 @@ Partition Value:    	[2008-01-01]
 Database:           	default             	 
 Table:              	anaylyze_external   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -284,8 +280,6 @@ Partition Value:    	[2008-01-01]
 Database:           	default             	 
 Table:              	anaylyze_external   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/stats_only_null.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/stats_only_null.q.out b/ql/src/test/results/clientpositive/stats_only_null.q.out
index f99b85a..55c5970 100644
--- a/ql/src/test/results/clientpositive/stats_only_null.q.out
+++ b/ql/src/test/results/clientpositive/stats_only_null.q.out
@@ -217,8 +217,6 @@ Partition Value:    	[2010]
 Database:           	default             	 
 Table:              	stats_null_part     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -260,8 +258,6 @@ Partition Value:    	[2011]
 Database:           	default             	 
 Table:              	stats_null_part     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/stats_partscan_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/stats_partscan_1.q.out b/ql/src/test/results/clientpositive/stats_partscan_1.q.out
index b15fbc7..e2a3bbd 100644
--- a/ql/src/test/results/clientpositive/stats_partscan_1.q.out
+++ b/ql/src/test/results/clientpositive/stats_partscan_1.q.out
@@ -74,8 +74,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart_partial_scan	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   
@@ -149,8 +147,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart_partial_scan	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -191,8 +187,6 @@ Partition Value:    	[2008-04-09, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart_partial_scan	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   


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

Posted by ga...@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


[07/50] [abbrv] hive git commit: HIVE-11174: Hive does not treat floating point signed zeros as equal (-0.0 should equal 0.0 according to IEEE floating point spec) (Sergio Pena, reviewed by Xuefu Zhang)

Posted by ga...@apache.org.
HIVE-11174: Hive does not treat floating point signed zeros as equal (-0.0 should equal 0.0 according to IEEE floating point spec) (Sergio Pena, reviewed by Xuefu Zhang)


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

Branch: refs/heads/hbase-metastore
Commit: af4aeab9c0dffc5f8e42428bf8b835dccc8771ef
Parents: 6e0d480
Author: Sergio Pena <se...@cloudera.com>
Authored: Wed Jul 15 09:47:06 2015 -0500
Committer: Sergio Pena <se...@cloudera.com>
Committed: Wed Jul 15 09:47:06 2015 -0500

----------------------------------------------------------------------
 .../objectinspector/ObjectInspectorUtils.java   | 18 ++++++++++++++--
 .../TestObjectInspectorUtils.java               | 22 ++++++++++++++++++++
 2 files changed, 38 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/af4aeab9/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java
index 041d218..6ef9f5d 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java
@@ -722,12 +722,26 @@ public final class ObjectInspectorUtils {
       case FLOAT: {
         float v1 = ((FloatObjectInspector) poi1).get(o1);
         float v2 = ((FloatObjectInspector) poi2).get(o2);
-        return Float.compare(v1, v2);
+
+        // The IEEE 754 floating point spec specifies that signed -0.0 and 0.0 should be treated as equal.
+        if (v1 == 0.0f && v2 == 0.0f) {
+          return 0;
+        } else {
+          // Float.compare() treats -0.0 and 0.0 as different
+          return Float.compare(v1, v2);
+        }
       }
       case DOUBLE: {
         double v1 = ((DoubleObjectInspector) poi1).get(o1);
         double v2 = ((DoubleObjectInspector) poi2).get(o2);
-        return Double.compare(v1, v2);
+
+        // The IEEE 754 floating point spec specifies that signed -0.0 and 0.0 should be treated as equal.
+        if (v1 == 0.0d && v2 == 0.0d) {
+          return 0;
+        } else {
+          // Double.compare() treats -0.0 and 0.0 as different
+          return Double.compare(v1, v2);
+        }
       }
       case STRING: {
         if (poi1.preferWritable() || poi2.preferWritable()) {

http://git-wip-us.apache.org/repos/asf/hive/blob/af4aeab9/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestObjectInspectorUtils.java
----------------------------------------------------------------------
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestObjectInspectorUtils.java b/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestObjectInspectorUtils.java
index f3fd6fa..ade0ef7 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestObjectInspectorUtils.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/objectinspector/TestObjectInspectorUtils.java
@@ -34,6 +34,28 @@ import org.apache.hadoop.hive.serde2.thrift.test.IntString;
  */
 public class TestObjectInspectorUtils extends TestCase {
 
+  public void testCompareFloatingNumberSignedZero() {
+    PrimitiveObjectInspector doubleOI = PrimitiveObjectInspectorFactory
+        .getPrimitiveJavaObjectInspector(PrimitiveObjectInspector.PrimitiveCategory.DOUBLE);
+
+    Double d1 = Double.valueOf("0.0");
+    Double d2 = Double.valueOf("-0.0");
+    assertEquals(0, ObjectInspectorUtils.compare(d1, doubleOI, d2, doubleOI));
+    assertEquals(0, ObjectInspectorUtils.compare(d2, doubleOI, d1, doubleOI));
+    assertEquals(0, ObjectInspectorUtils.compare(d1, doubleOI, d1, doubleOI));
+    assertEquals(0, ObjectInspectorUtils.compare(d2, doubleOI, d2, doubleOI));
+
+    PrimitiveObjectInspector floatOI = PrimitiveObjectInspectorFactory
+        .getPrimitiveJavaObjectInspector(PrimitiveObjectInspector.PrimitiveCategory.FLOAT);
+
+    Float f1 = Float.valueOf("0.0");
+    Float f2 = Float.valueOf("-0.0");
+    assertEquals(0, ObjectInspectorUtils.compare(f1, floatOI, f2, floatOI));
+    assertEquals(0, ObjectInspectorUtils.compare(f2, floatOI, f1, floatOI));
+    assertEquals(0, ObjectInspectorUtils.compare(f1, floatOI, f1, floatOI));
+    assertEquals(0, ObjectInspectorUtils.compare(f2, floatOI, f2, floatOI));
+  }
+
   public void testObjectInspectorUtils() throws Throwable {
     try {
       ObjectInspector oi1 = ObjectInspectorFactory


[22/50] [abbrv] hive git commit: HIVE-11024: Error inserting a date value via parameter marker (PreparedStatement.setDate) (Yongzhi Chen, reviewed by Sergio Pena)

Posted by ga...@apache.org.
HIVE-11024: Error inserting a date value via parameter marker (PreparedStatement.setDate) (Yongzhi Chen, reviewed by Sergio Pena)


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

Branch: refs/heads/hbase-metastore
Commit: e74dc320ed4c2cae0ab34cf89fb695c1a5f2e31f
Parents: e2ee458
Author: Sergio Pena <se...@cloudera.com>
Authored: Thu Jul 16 07:28:16 2015 -0500
Committer: Sergio Pena <se...@cloudera.com>
Committed: Thu Jul 16 07:28:16 2015 -0500

----------------------------------------------------------------------
 .../java/org/apache/hive/jdbc/TestJdbcDriver2.java  | 16 ++++++++++++++++
 .../org/apache/hive/jdbc/HivePreparedStatement.java |  2 +-
 2 files changed, 17 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/e74dc320/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java
index e4b9cd7..b2dd2ab 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java
@@ -2366,4 +2366,20 @@ public void testParseUrlHttpMode() throws SQLException, JdbcUriParseException,
       assertTrue(accumulatedLogs.contains(expectedLog));
     }
   }
+  @Test
+  public void testPrepareSetDate() throws Exception {
+    try {
+      String sql = "select * from " + dataTypeTableName + " where c20 = ?";
+      PreparedStatement ps = con.prepareStatement(sql);
+      java.sql.Date dtValue = java.sql.Date.valueOf("2013-01-01");
+      ps.setDate(1, dtValue);
+      ResultSet res = ps.executeQuery();
+      assertTrue(res.next());
+      assertEquals("2013-01-01", res.getString(20));
+      ps.close();
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.toString());
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/e74dc320/jdbc/src/java/org/apache/hive/jdbc/HivePreparedStatement.java
----------------------------------------------------------------------
diff --git a/jdbc/src/java/org/apache/hive/jdbc/HivePreparedStatement.java b/jdbc/src/java/org/apache/hive/jdbc/HivePreparedStatement.java
index 2625155..7687537 100644
--- a/jdbc/src/java/org/apache/hive/jdbc/HivePreparedStatement.java
+++ b/jdbc/src/java/org/apache/hive/jdbc/HivePreparedStatement.java
@@ -436,7 +436,7 @@ public class HivePreparedStatement extends HiveStatement implements PreparedStat
    */
 
   public void setDate(int parameterIndex, Date x) throws SQLException {
-    this.parameters.put(parameterIndex, x.toString());
+    this.parameters.put(parameterIndex, "'" + x.toString() + "'");
   }
 
   /*


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

Posted by ga...@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


[19/50] [abbrv] hive git commit: HIVE-11145 Remove OFFLINE and NO_DROP from tables and partitions (gates, reviewed by Ashutosh Chauhan)

Posted by ga...@apache.org.
HIVE-11145 Remove OFFLINE and NO_DROP from tables and partitions (gates, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/hbase-metastore
Commit: d6ec52ee094d94377442d96d450575462a9497b7
Parents: 7338d8e
Author: Alan Gates <ga...@hortonworks.com>
Authored: Wed Jul 15 17:23:23 2015 -0700
Committer: Alan Gates <ga...@hortonworks.com>
Committed: Wed Jul 15 17:23:23 2015 -0700

----------------------------------------------------------------------
 .../results/positive/external_table_ppd.q.out   |   1 -
 .../positive/hbase_binary_storage_queries.q.out |   2 -
 .../src/test/results/positive/hbase_stats.q.out |   7 --
 .../test/results/positive/hbase_stats2.q.out    |   7 --
 .../test/results/positive/hbase_stats3.q.out    |  12 --
 .../positive/hbase_stats_empty_partition.q.out  |   2 -
 .../SemanticAnalysis/HCatSemanticAnalyzer.java  |   7 +-
 .../hive/hcatalog/api/HCatClientHMSImpl.java    |  14 +--
 .../hadoop/hive/metastore/HiveMetaStore.java    |  11 +-
 .../hive/metastore/HiveMetaStoreClient.java     |   7 +-
 .../hadoop/hive/metastore/IMetaStoreClient.java |  57 +++++-----
 .../hadoop/hive/metastore/MetaStoreUtils.java   |  26 -----
 .../hive/metastore/PartitionDropOptions.java    |   6 -
 .../hadoop/hive/metastore/ProtectMode.java      |  97 ----------------
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |  50 ---------
 .../hadoop/hive/ql/hooks/WriteEntity.java       |   6 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |  57 +++++-----
 .../hadoop/hive/ql/metadata/Partition.java      |  51 ---------
 .../apache/hadoop/hive/ql/metadata/Table.java   |  65 ++---------
 .../formatting/MetaDataFormatUtils.java         |  24 ++--
 .../hive/ql/parse/DDLSemanticAnalyzer.java      | 112 +++++--------------
 .../apache/hadoop/hive/ql/parse/EximUtil.java   |  31 ++---
 .../org/apache/hadoop/hive/ql/parse/HiveLexer.g |   5 -
 .../apache/hadoop/hive/ql/parse/HiveParser.g    |  38 +------
 .../hive/ql/parse/LoadSemanticAnalyzer.java     |  11 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |  42 +------
 .../hive/ql/parse/SemanticAnalyzerFactory.java  |   8 +-
 .../hadoop/hive/ql/plan/AlterTableDesc.java     |  20 ++--
 .../hadoop/hive/ql/plan/DropTableDesc.java      |  26 +----
 .../hadoop/hive/ql/plan/HiveOperation.java      |   2 -
 .../alter_partition_invalidspec.q               |   8 --
 .../clientnegative/alter_partition_nodrop.q     |   9 --
 .../alter_partition_nodrop_table.q              |   9 --
 .../clientnegative/alter_partition_offline.q    |  11 --
 .../clientnegative/drop_table_failure3.q        |  12 --
 .../queries/clientnegative/protectmode_part.q   |  15 ---
 .../queries/clientnegative/protectmode_part1.q  |  21 ----
 .../queries/clientnegative/protectmode_part2.q  |   9 --
 .../clientnegative/protectmode_part_no_drop.q   |  10 --
 .../clientnegative/protectmode_part_no_drop2.q  |  11 --
 .../queries/clientnegative/protectmode_tbl1.q   |   8 --
 .../queries/clientnegative/protectmode_tbl2.q   |  12 --
 .../queries/clientnegative/protectmode_tbl3.q   |  10 --
 .../queries/clientnegative/protectmode_tbl4.q   |  15 ---
 .../queries/clientnegative/protectmode_tbl5.q   |  15 ---
 .../queries/clientnegative/protectmode_tbl6.q   |   8 --
 .../queries/clientnegative/protectmode_tbl7.q   |  13 ---
 .../queries/clientnegative/protectmode_tbl8.q   |  13 ---
 .../clientnegative/protectmode_tbl_no_drop.q    |   9 --
 .../test/queries/clientnegative/sa_fail_hook3.q |   4 -
 .../alter_partition_protect_mode.q              |  26 -----
 .../drop_partitions_ignore_protection.q         |  10 --
 .../test/queries/clientpositive/protectmode.q   |  63 -----------
 .../test/queries/clientpositive/protectmode2.q  |  23 ----
 .../alter_numbuckets_partitioned_table.q.out    |   8 --
 .../results/beelinepositive/create_like.q.out   |   3 -
 .../results/beelinepositive/create_like2.q.out  |   1 -
 .../beelinepositive/create_like_view.q.out      |   4 -
 .../beelinepositive/create_skewed_table1.q.out  |   3 -
 .../results/beelinepositive/create_view.q.out   |  14 ---
 .../create_view_partitioned.q.out               |   3 -
 ql/src/test/results/beelinepositive/ctas.q.out  |   5 -
 .../describe_formatted_view_partitioned.q.out   |   1 -
 .../beelinepositive/describe_table.q.out        |   3 -
 .../test/results/beelinepositive/merge3.q.out   |   1 -
 .../part_inherit_tbl_props.q.out                |   1 -
 .../part_inherit_tbl_props_empty.q.out          |   1 -
 .../part_inherit_tbl_props_with_star.q.out      |   1 -
 .../results/beelinepositive/protectmode2.q.out  |   2 -
 .../test/results/beelinepositive/stats1.q.out   |   2 -
 .../test/results/beelinepositive/stats10.q.out  |   3 -
 .../test/results/beelinepositive/stats11.q.out  |   4 -
 .../test/results/beelinepositive/stats12.q.out  |   5 -
 .../test/results/beelinepositive/stats13.q.out  |   6 -
 .../test/results/beelinepositive/stats14.q.out  |   5 -
 .../test/results/beelinepositive/stats15.q.out  |   5 -
 .../test/results/beelinepositive/stats16.q.out  |   2 -
 .../test/results/beelinepositive/stats18.q.out  |   2 -
 .../test/results/beelinepositive/stats2.q.out   |   2 -
 .../test/results/beelinepositive/stats3.q.out   |   2 -
 .../test/results/beelinepositive/stats4.q.out   |   6 -
 .../test/results/beelinepositive/stats5.q.out   |   1 -
 .../test/results/beelinepositive/stats6.q.out   |   5 -
 .../test/results/beelinepositive/stats7.q.out   |   3 -
 .../test/results/beelinepositive/stats8.q.out   |  10 --
 .../test/results/beelinepositive/stats9.q.out   |   1 -
 .../beelinepositive/stats_empty_partition.q.out |   1 -
 .../clientnegative/alter_file_format.q.out      |   1 -
 .../alter_view_as_select_with_partition.q.out   |   1 -
 .../stats_partialscan_autogether.q.out          |   2 -
 .../clientpositive/alter_file_format.q.out      |  19 ----
 .../clientpositive/alter_merge_stats_orc.q.out  |   8 --
 .../alter_numbuckets_partitioned_table.q.out    |  16 ---
 .../alter_numbuckets_partitioned_table2.q.out   |  27 -----
 ...lter_numbuckets_partitioned_table2_h23.q.out |  27 -----
 ...alter_numbuckets_partitioned_table_h23.q.out |  16 ---
 .../alter_partition_clusterby_sortby.q.out      |   7 --
 .../clientpositive/alter_skewed_table.q.out     |   6 -
 .../clientpositive/alter_table_not_sorted.q.out |   2 -
 .../clientpositive/alter_table_serde2.q.out     |   6 -
 .../clientpositive/alter_view_as_select.q.out   |   3 -
 .../clientpositive/authorization_index.q.out    |   1 -
 .../test/results/clientpositive/bucket5.q.out   |   1 -
 .../create_alter_list_bucketing_table1.q.out    |   7 --
 .../results/clientpositive/create_like.q.out    |   9 --
 .../results/clientpositive/create_like2.q.out   |   1 -
 .../clientpositive/create_like_tbl_props.q.out  |   5 -
 .../clientpositive/create_like_view.q.out       |   4 -
 .../clientpositive/create_or_replace_view.q.out |   5 -
 .../clientpositive/create_skewed_table1.q.out   |   3 -
 .../results/clientpositive/create_view.q.out    |  14 ---
 .../create_view_partitioned.q.out               |   3 -
 .../clientpositive/create_view_translate.q.out  |   2 -
 ql/src/test/results/clientpositive/ctas.q.out   |   5 -
 .../results/clientpositive/ctas_colname.q.out   |   7 --
 .../results/clientpositive/ctas_hadoop20.q.out  |   5 -
 .../ctas_uses_database_location.q.out           |   1 -
 .../clientpositive/database_location.q.out      |   2 -
 .../results/clientpositive/decimal_serde.q.out  |   2 -
 .../clientpositive/default_file_format.q.out    |   5 -
 .../describe_comment_indent.q.out               |   1 -
 .../describe_comment_nonascii.q.out             |   1 -
 .../describe_formatted_view_partitioned.q.out   |   2 -
 .../clientpositive/describe_syntax.q.out        |   6 -
 .../results/clientpositive/describe_table.q.out |   7 --
 .../dynpart_sort_opt_vectorization.q.out        |  32 ------
 .../dynpart_sort_optimization.q.out             |  32 ------
 .../dynpart_sort_optimization2.q.out            |  24 ----
 .../encrypted/encryption_insert_values.q.out    |   1 -
 .../clientpositive/exim_hidden_files.q.out      |   1 -
 .../clientpositive/index_skewtable.q.out        |   1 -
 .../clientpositive/infer_bucket_sort.q.out      |  50 ---------
 .../infer_bucket_sort_bucketed_table.q.out      |   2 -
 .../infer_bucket_sort_convert_join.q.out        |   4 -
 .../infer_bucket_sort_dyn_part.q.out            |  16 ---
 .../infer_bucket_sort_grouping_operators.q.out  |  12 --
 .../infer_bucket_sort_list_bucket.q.out         |   4 -
 .../infer_bucket_sort_map_operators.q.out       |   8 --
 .../infer_bucket_sort_merge.q.out               |   4 -
 .../infer_bucket_sort_multi_insert.q.out        |  16 ---
 .../infer_bucket_sort_num_buckets.q.out         |   4 -
 .../infer_bucket_sort_reducers_power_two.q.out  |  12 --
 .../results/clientpositive/lb_fs_stats.q.out    |   2 -
 .../clientpositive/list_bucket_dml_1.q.out      |   4 -
 .../list_bucket_dml_10.q.java1.7.out            |   2 -
 .../list_bucket_dml_10.q.java1.8.out            |   2 -
 .../list_bucket_dml_11.q.java1.7.out            |   2 -
 .../list_bucket_dml_11.q.java1.8.out            |   2 -
 .../list_bucket_dml_12.q.java1.7.out            |   2 -
 .../list_bucket_dml_12.q.java1.8.out            |   2 -
 .../list_bucket_dml_13.q.java1.7.out            |   2 -
 .../list_bucket_dml_13.q.java1.8.out            |   2 -
 .../clientpositive/list_bucket_dml_14.q.out     |   1 -
 .../list_bucket_dml_2.q.java1.7.out             |   2 -
 .../list_bucket_dml_2.q.java1.8.out             |   2 -
 .../clientpositive/list_bucket_dml_3.q.out      |   2 -
 .../list_bucket_dml_4.q.java1.7.out             |   4 -
 .../list_bucket_dml_4.q.java1.8.out             |   4 -
 .../list_bucket_dml_5.q.java1.7.out             |   4 -
 .../list_bucket_dml_5.q.java1.8.out             |   4 -
 .../list_bucket_dml_6.q.java1.7.out             |   8 --
 .../list_bucket_dml_6.q.java1.8.out             |   8 --
 .../clientpositive/list_bucket_dml_7.q.out      |   8 --
 .../list_bucket_dml_8.q.java1.7.out             |   6 -
 .../list_bucket_dml_8.q.java1.8.out             |   6 -
 .../list_bucket_dml_9.q.java1.7.out             |   4 -
 .../list_bucket_dml_9.q.java1.8.out             |   4 -
 .../list_bucket_query_multiskew_1.q.out         |   2 -
 .../list_bucket_query_multiskew_2.q.out         |   2 -
 .../list_bucket_query_multiskew_3.q.out         |   6 -
 .../list_bucket_query_oneskew_1.q.out           |   2 -
 .../list_bucket_query_oneskew_2.q.out           |   2 -
 .../list_bucket_query_oneskew_3.q.out           |   2 -
 ql/src/test/results/clientpositive/merge3.q.out |   1 -
 .../results/clientpositive/orc_analyze.q.out    |  48 --------
 .../results/clientpositive/orc_create.q.out     |   6 -
 .../clientpositive/parallel_orderby.q.out       |   2 -
 .../parquet_array_null_element.q.out            |   1 -
 .../results/clientpositive/parquet_create.q.out |   1 -
 .../clientpositive/parquet_partitioned.q.out    |   1 -
 .../results/clientpositive/parquet_serde.q.out  |   5 -
 .../clientpositive/part_inherit_tbl_props.q.out |   2 -
 .../part_inherit_tbl_props_empty.q.out          |   2 -
 .../part_inherit_tbl_props_with_star.q.out      |   2 -
 .../partition_coltype_literals.q.out            |  16 ---
 .../results/clientpositive/protectmode2.q.out   |   2 -
 .../clientpositive/rcfile_default_format.q.out  |   8 --
 .../clientpositive/selectDistinctStar.q.out     |   2 -
 .../spark/alter_merge_stats_orc.q.out           |   8 --
 .../results/clientpositive/spark/bucket5.q.out  |   1 -
 .../results/clientpositive/spark/ctas.q.out     |   5 -
 .../infer_bucket_sort_bucketed_table.q.out      |   2 -
 .../spark/infer_bucket_sort_convert_join.q.out  |   4 -
 .../spark/infer_bucket_sort_map_operators.q.out |   8 --
 .../spark/infer_bucket_sort_merge.q.out         |   4 -
 .../spark/infer_bucket_sort_num_buckets.q.out   |   4 -
 .../infer_bucket_sort_reducers_power_two.q.out  |  12 --
 .../spark/list_bucket_dml_10.q.java1.7.out      |   2 -
 .../spark/list_bucket_dml_10.q.java1.8.out      |   2 -
 .../spark/list_bucket_dml_2.q.java1.7.out       |   2 -
 .../spark/list_bucket_dml_2.q.java1.8.out       |   2 -
 .../spark/list_bucket_dml_2.q.out               | Bin 28747 -> 28667 bytes
 .../clientpositive/spark/orc_analyze.q.out      |  22 ----
 .../clientpositive/spark/parallel_orderby.q.out |   2 -
 .../results/clientpositive/spark/stats1.q.out   |   2 -
 .../results/clientpositive/spark/stats10.q.out  |   5 -
 .../results/clientpositive/spark/stats12.q.out  |   9 --
 .../results/clientpositive/spark/stats13.q.out  |  10 --
 .../results/clientpositive/spark/stats14.q.out  |   7 --
 .../results/clientpositive/spark/stats15.q.out  |   7 --
 .../results/clientpositive/spark/stats16.q.out  |   2 -
 .../results/clientpositive/spark/stats18.q.out  |   4 -
 .../results/clientpositive/spark/stats2.q.out   |   2 -
 .../results/clientpositive/spark/stats20.q.out  |   2 -
 .../results/clientpositive/spark/stats3.q.out   |   2 -
 .../results/clientpositive/spark/stats5.q.out   |   1 -
 .../results/clientpositive/spark/stats6.q.out   |   9 --
 .../results/clientpositive/spark/stats7.q.out   |   5 -
 .../results/clientpositive/spark/stats8.q.out   |  18 ---
 .../results/clientpositive/spark/stats9.q.out   |   1 -
 .../clientpositive/spark/stats_counter.q.out    |   2 -
 .../spark/stats_counter_partitioned.q.out       |  16 ---
 .../clientpositive/spark/stats_noscan_1.q.out   |  17 ---
 .../clientpositive/spark/stats_noscan_2.q.out   |   6 -
 .../clientpositive/spark/stats_only_null.q.out  |   4 -
 .../spark/stats_partscan_1_23.q.out             |   6 -
 .../results/clientpositive/spark/statsfs.q.out  |  14 ---
 .../clientpositive/spark/union_remove_1.q.out   |   1 -
 .../clientpositive/spark/union_remove_10.q.out  |   1 -
 .../clientpositive/spark/union_remove_11.q.out  |   1 -
 .../clientpositive/spark/union_remove_12.q.out  |   1 -
 .../clientpositive/spark/union_remove_13.q.out  |   1 -
 .../clientpositive/spark/union_remove_14.q.out  |   1 -
 .../clientpositive/spark/union_remove_15.q.out  |   1 -
 .../clientpositive/spark/union_remove_16.q.out  |   1 -
 .../clientpositive/spark/union_remove_17.q.out  |   1 -
 .../clientpositive/spark/union_remove_18.q.out  |   1 -
 .../clientpositive/spark/union_remove_19.q.out  |   1 -
 .../clientpositive/spark/union_remove_2.q.out   |   1 -
 .../clientpositive/spark/union_remove_20.q.out  |   1 -
 .../clientpositive/spark/union_remove_21.q.out  |   1 -
 .../clientpositive/spark/union_remove_22.q.out  |   1 -
 .../clientpositive/spark/union_remove_23.q.out  |   1 -
 .../clientpositive/spark/union_remove_24.q.out  |   1 -
 .../clientpositive/spark/union_remove_25.q.out  |   6 -
 .../clientpositive/spark/union_remove_3.q.out   |   1 -
 .../clientpositive/spark/union_remove_4.q.out   |   1 -
 .../clientpositive/spark/union_remove_5.q.out   |   1 -
 .../clientpositive/spark/union_remove_7.q.out   |   1 -
 .../clientpositive/spark/union_remove_8.q.out   |   1 -
 .../clientpositive/spark/union_remove_9.q.out   |   1 -
 ql/src/test/results/clientpositive/stats1.q.out |   2 -
 .../test/results/clientpositive/stats10.q.out   |   5 -
 .../test/results/clientpositive/stats11.q.out   |   8 --
 .../test/results/clientpositive/stats12.q.out   |   9 --
 .../test/results/clientpositive/stats13.q.out   |  10 --
 .../test/results/clientpositive/stats14.q.out   |   7 --
 .../test/results/clientpositive/stats15.q.out   |   7 --
 .../test/results/clientpositive/stats16.q.out   |   2 -
 .../test/results/clientpositive/stats18.q.out   |   4 -
 .../test/results/clientpositive/stats19.q.out   |  12 --
 ql/src/test/results/clientpositive/stats2.q.out |   2 -
 .../test/results/clientpositive/stats20.q.out   |   2 -
 ql/src/test/results/clientpositive/stats3.q.out |   2 -
 ql/src/test/results/clientpositive/stats4.q.out |  10 --
 ql/src/test/results/clientpositive/stats5.q.out |   1 -
 ql/src/test/results/clientpositive/stats6.q.out |   9 --
 ql/src/test/results/clientpositive/stats7.q.out |   5 -
 ql/src/test/results/clientpositive/stats8.q.out |  18 ---
 ql/src/test/results/clientpositive/stats9.q.out |   1 -
 .../results/clientpositive/stats_counter.q.out  |   2 -
 .../stats_counter_partitioned.q.out             |  16 ---
 .../clientpositive/stats_empty_partition.q.out  |   2 -
 .../clientpositive/stats_invalidation.q.out     |   2 -
 .../stats_list_bucket.q.java1.7.out             |   3 -
 .../stats_list_bucket.q.java1.8.out             |   3 -
 .../results/clientpositive/stats_noscan_1.q.out |  17 ---
 .../results/clientpositive/stats_noscan_2.q.out |   6 -
 .../clientpositive/stats_only_null.q.out        |   4 -
 .../clientpositive/stats_partscan_1.q.out       |   6 -
 .../clientpositive/stats_partscan_1_23.q.out    |   6 -
 .../test/results/clientpositive/statsfs.q.out   |  14 ---
 .../tez/alter_merge_stats_orc.q.out             |   8 --
 .../test/results/clientpositive/tez/ctas.q.out  |   5 -
 .../tez/dynpart_sort_opt_vectorization.q.out    |  32 ------
 .../tez/dynpart_sort_optimization.q.out         |  32 ------
 .../tez/dynpart_sort_optimization2.q.out        |  24 ----
 .../clientpositive/tez/orc_analyze.q.out        |  48 --------
 .../clientpositive/tez/selectDistinctStar.q.out |   2 -
 .../clientpositive/tez/stats_counter.q.out      |   2 -
 .../tez/stats_counter_partitioned.q.out         |  16 ---
 .../clientpositive/tez/stats_noscan_1.q.out     |  17 ---
 .../clientpositive/tez/stats_only_null.q.out    |   4 -
 .../results/clientpositive/tez/tez_fsstat.q.out |   2 -
 .../clientpositive/truncate_column.q.out        |  11 --
 .../clientpositive/unicode_notation.q.out       |   3 -
 .../results/clientpositive/union_remove_1.q.out |   1 -
 .../clientpositive/union_remove_10.q.out        |   1 -
 .../clientpositive/union_remove_11.q.out        |   1 -
 .../clientpositive/union_remove_12.q.out        |   1 -
 .../clientpositive/union_remove_13.q.out        |   1 -
 .../clientpositive/union_remove_14.q.out        |   1 -
 .../clientpositive/union_remove_15.q.out        |   1 -
 .../clientpositive/union_remove_16.q.out        |   1 -
 .../clientpositive/union_remove_17.q.out        |   1 -
 .../clientpositive/union_remove_18.q.out        |   1 -
 .../clientpositive/union_remove_19.q.out        |   1 -
 .../results/clientpositive/union_remove_2.q.out |   1 -
 .../clientpositive/union_remove_20.q.out        |   1 -
 .../clientpositive/union_remove_21.q.out        |   1 -
 .../clientpositive/union_remove_22.q.out        |   1 -
 .../clientpositive/union_remove_23.q.out        |   1 -
 .../clientpositive/union_remove_24.q.out        |   1 -
 .../clientpositive/union_remove_25.q.out        |   6 -
 .../results/clientpositive/union_remove_3.q.out |   1 -
 .../results/clientpositive/union_remove_4.q.out |   1 -
 .../results/clientpositive/union_remove_5.q.out |   1 -
 .../results/clientpositive/union_remove_7.q.out |   1 -
 .../results/clientpositive/union_remove_8.q.out |   1 -
 .../results/clientpositive/union_remove_9.q.out |   1 -
 320 files changed, 150 insertions(+), 2473 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/hbase-handler/src/test/results/positive/external_table_ppd.q.out
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/results/positive/external_table_ppd.q.out b/hbase-handler/src/test/results/positive/external_table_ppd.q.out
index 57424ce..83eb2f5 100644
--- a/hbase-handler/src/test/results/positive/external_table_ppd.q.out
+++ b/hbase-handler/src/test/results/positive/external_table_ppd.q.out
@@ -52,7 +52,6 @@ boolean_col         	boolean             	from deserializer
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/hbase-handler/src/test/results/positive/hbase_binary_storage_queries.q.out
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/results/positive/hbase_binary_storage_queries.q.out b/hbase-handler/src/test/results/positive/hbase_binary_storage_queries.q.out
index 578ddb2..f212331 100644
--- a/hbase-handler/src/test/results/positive/hbase_binary_storage_queries.q.out
+++ b/hbase-handler/src/test/results/positive/hbase_binary_storage_queries.q.out
@@ -52,7 +52,6 @@ boolean_col         	boolean             	from deserializer
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -228,7 +227,6 @@ boolean_col         	boolean             	from deserializer
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	EXTERNAL_TABLE      	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/hbase-handler/src/test/results/positive/hbase_stats.q.out
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/results/positive/hbase_stats.q.out b/hbase-handler/src/test/results/positive/hbase_stats.q.out
index f12b136..f34720d 100644
--- a/hbase-handler/src/test/results/positive/hbase_stats.q.out
+++ b/hbase-handler/src/test/results/positive/hbase_stats.q.out
@@ -38,7 +38,6 @@ value               	string              	default
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -142,7 +141,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -181,8 +179,6 @@ Partition Value:    	[2010-04-08, 11]
 Database:           	default             	 
 Table:              	stats_part          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -223,8 +219,6 @@ Partition Value:    	[2010-04-08, 12]
 Database:           	default             	 
 Table:              	stats_part          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -283,7 +277,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/hbase-handler/src/test/results/positive/hbase_stats2.q.out
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/results/positive/hbase_stats2.q.out b/hbase-handler/src/test/results/positive/hbase_stats2.q.out
index a60dee2..aad2e3a 100644
--- a/hbase-handler/src/test/results/positive/hbase_stats2.q.out
+++ b/hbase-handler/src/test/results/positive/hbase_stats2.q.out
@@ -38,7 +38,6 @@ value               	string              	default
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -142,7 +141,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -181,8 +179,6 @@ Partition Value:    	[2010-04-08, 11]
 Database:           	default             	 
 Table:              	stats_part          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -223,8 +219,6 @@ Partition Value:    	[2010-04-08, 12]
 Database:           	default             	 
 Table:              	stats_part          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -283,7 +277,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/hbase-handler/src/test/results/positive/hbase_stats3.q.out
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/results/positive/hbase_stats3.q.out b/hbase-handler/src/test/results/positive/hbase_stats3.q.out
index 114847c..063800f 100644
--- a/hbase-handler/src/test/results/positive/hbase_stats3.q.out
+++ b/hbase-handler/src/test/results/positive/hbase_stats3.q.out
@@ -40,8 +40,6 @@ Partition Value:    	[2010-04-08, 13]
 Database:           	default             	 
 Table:              	stats_part          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -94,8 +92,6 @@ Partition Value:    	[2010-04-08, 13]
 Database:           	default             	 
 Table:              	stats_part          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -148,8 +144,6 @@ Partition Value:    	[2010-04-08, 13]
 Database:           	default             	 
 Table:              	stats_part          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -206,8 +200,6 @@ Partition Value:    	[2010-04-08, 13]
 Database:           	default             	 
 Table:              	stats_part          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -260,8 +252,6 @@ Partition Value:    	[2010-04-08, 13]
 Database:           	default             	 
 Table:              	stats_part          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -314,8 +304,6 @@ Partition Value:    	[2010-04-08, 13]
 Database:           	default             	 
 Table:              	stats_part          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/hbase-handler/src/test/results/positive/hbase_stats_empty_partition.q.out
----------------------------------------------------------------------
diff --git a/hbase-handler/src/test/results/positive/hbase_stats_empty_partition.q.out b/hbase-handler/src/test/results/positive/hbase_stats_empty_partition.q.out
index 65e0a6f..c13817e 100644
--- a/hbase-handler/src/test/results/positive/hbase_stats_empty_partition.q.out
+++ b/hbase-handler/src/test/results/positive/hbase_stats_empty_partition.q.out
@@ -43,8 +43,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	tmptable            	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/hcatalog/core/src/main/java/org/apache/hive/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzer.java b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzer.java
index 11d0743..18bf172 100644
--- a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzer.java
+++ b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzer.java
@@ -18,9 +18,6 @@
  */
 package org.apache.hive.hcatalog.cli.SemanticAnalysis;
 
-import java.io.Serializable;
-import java.util.List;
-
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.Utilities;
@@ -50,6 +47,9 @@ import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hive.hcatalog.common.ErrorType;
 import org.apache.hive.hcatalog.common.HCatException;
 
+import java.io.Serializable;
+import java.util.List;
+
 public class HCatSemanticAnalyzer extends HCatSemanticAnalyzerBase {
 
   private AbstractSemanticAnalyzerHook hook;
@@ -237,7 +237,6 @@ public class HCatSemanticAnalyzer extends HCatSemanticAnalyzerBase {
           case HiveParser.TOK_ALTERTABLE_EXCHANGEPARTITION:
           case HiveParser.TOK_ALTERTABLE_SKEWED:
           case HiveParser.TOK_ALTERTABLE_FILEFORMAT:
-          case HiveParser.TOK_ALTERTABLE_PROTECTMODE:
           case HiveParser.TOK_ALTERTABLE_LOCATION:
           case HiveParser.TOK_ALTERTABLE_MERGEFILES:
           case HiveParser.TOK_ALTERTABLE_RENAMEPART:

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/hcatalog/webhcat/java-client/src/main/java/org/apache/hive/hcatalog/api/HCatClientHMSImpl.java
----------------------------------------------------------------------
diff --git a/hcatalog/webhcat/java-client/src/main/java/org/apache/hive/hcatalog/api/HCatClientHMSImpl.java b/hcatalog/webhcat/java-client/src/main/java/org/apache/hive/hcatalog/api/HCatClientHMSImpl.java
index 3a69581..41571fc 100644
--- a/hcatalog/webhcat/java-client/src/main/java/org/apache/hive/hcatalog/api/HCatClientHMSImpl.java
+++ b/hcatalog/webhcat/java-client/src/main/java/org/apache/hive/hcatalog/api/HCatClientHMSImpl.java
@@ -18,13 +18,6 @@
  */
 package org.apache.hive.hcatalog.api;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
 import com.google.common.base.Function;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
@@ -76,6 +69,12 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import javax.annotation.Nullable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
 
 /**
  * The HCatClientHMSImpl is the Hive Metastore client based implementation of
@@ -588,7 +587,6 @@ public class HCatClientHMSImpl extends HCatClient {
             Utilities.serializeExpressionToKryo(partitionExpression));
     hmsClient.dropPartitions(table.getDbName(), table.getTableName(), Arrays.asList(serializedPartitionExpression),
         deleteData && !isExternal(table),  // Delete data?
-        false,                             // Ignore Protection?
         ifExists,                          // Fail if table doesn't exist?
         false);                            // Need results back?
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/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 2ef5aa0..0edf11f 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -230,7 +230,9 @@ import java.util.concurrent.locks.ReentrantLock;
 import java.util.regex.Pattern;
 
 import static org.apache.commons.lang.StringUtils.join;
-import static org.apache.hadoop.hive.metastore.MetaStoreUtils.*;
+import static org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_COMMENT;
+import static org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME;
+import static org.apache.hadoop.hive.metastore.MetaStoreUtils.validateName;
 
 /**
  * TODO:pc remove application logic to a separate interface.
@@ -2774,10 +2776,9 @@ public class HiveMetaStore extends ThriftHiveMetastore {
         }
 
         for (Partition part : parts) {
-          if (!ignoreProtection && !MetaStoreUtils.canDropPartition(tbl, part)) {
-            throw new MetaException("Table " + tbl.getTableName()
-                + " Partition " + part + " is protected from being dropped");
-          }
+
+          // TODO - we need to speed this up for the normal path where all partitions are under
+          // the table and we don't have to stat every partition
 
           firePreEvent(new PreDropPartitionEvent(tbl, part, deleteData, this));
           if (colNames != null) {

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index a5f5053..66fbfe4 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -842,7 +842,6 @@ public class HiveMetaStoreClient implements IMetaStoreClient {
     rps.setExprs(exprs);
     DropPartitionsRequest req = new DropPartitionsRequest(dbName, tblName, rps);
     req.setDeleteData(options.deleteData);
-    req.setIgnoreProtection(options.ignoreProtection);
     req.setNeedResult(options.returnResults);
     req.setIfExists(options.ifExists);
     if (options.purgeData) {
@@ -854,13 +853,12 @@ public class HiveMetaStoreClient implements IMetaStoreClient {
 
   @Override
   public List<Partition> dropPartitions(String dbName, String tblName,
-      List<ObjectPair<Integer, byte[]>> partExprs, boolean deleteData, boolean ignoreProtection,
+      List<ObjectPair<Integer, byte[]>> partExprs, boolean deleteData,
       boolean ifExists, boolean needResult) throws NoSuchObjectException, MetaException, TException {
 
     return dropPartitions(dbName, tblName, partExprs,
                           PartitionDropOptions.instance()
                                               .deleteData(deleteData)
-                                              .ignoreProtection(ignoreProtection)
                                               .ifExists(ifExists)
                                               .returnResults(needResult));
 
@@ -868,13 +866,12 @@ public class HiveMetaStoreClient implements IMetaStoreClient {
 
   @Override
   public List<Partition> dropPartitions(String dbName, String tblName,
-      List<ObjectPair<Integer, byte[]>> partExprs, boolean deleteData, boolean ignoreProtection,
+      List<ObjectPair<Integer, byte[]>> partExprs, boolean deleteData,
       boolean ifExists) throws NoSuchObjectException, MetaException, TException {
     // By default, we need the results from dropPartitions();
     return dropPartitions(dbName, tblName, partExprs,
                           PartitionDropOptions.instance()
                                               .deleteData(deleteData)
-                                              .ignoreProtection(ignoreProtection)
                                               .ifExists(ifExists));
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java b/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
index 341b0ca..147ffcc 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
@@ -19,50 +19,30 @@
 package org.apache.hadoop.hive.metastore;
 
 
+import org.apache.hadoop.hive.common.ObjectPair;
 import org.apache.hadoop.hive.common.ValidTxnList;
 import org.apache.hadoop.hive.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.api.AddDynamicPartitions;
-import org.apache.hadoop.hive.metastore.api.CompactionType;
-import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
-import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
-import org.apache.hadoop.hive.metastore.api.FireEventRequest;
-import org.apache.hadoop.hive.metastore.api.FireEventResponse;
-import org.apache.hadoop.hive.metastore.api.GetOpenTxnsInfoResponse;
-import org.apache.hadoop.hive.metastore.api.HeartbeatTxnRangeResponse;
-import org.apache.hadoop.hive.metastore.api.LockRequest;
-import org.apache.hadoop.hive.metastore.api.LockResponse;
-import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
-import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
-import org.apache.hadoop.hive.metastore.api.NotificationEvent;
-import org.apache.hadoop.hive.metastore.api.NotificationEventResponse;
-import org.apache.hadoop.hive.metastore.api.OpenTxnsResponse;
-import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
-import org.apache.hadoop.hive.metastore.api.ShowLocksResponse;
-import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
-import org.apache.hadoop.hive.metastore.api.TxnOpenException;
-import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
-import org.apache.thrift.TException;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.hive.common.ObjectPair;
 import org.apache.hadoop.hive.common.classification.InterfaceAudience.Public;
 import org.apache.hadoop.hive.common.classification.InterfaceStability.Evolving;
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.AggrStats;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
+import org.apache.hadoop.hive.metastore.api.CompactionType;
 import org.apache.hadoop.hive.metastore.api.ConfigValSecurityException;
+import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.FireEventRequest;
+import org.apache.hadoop.hive.metastore.api.FireEventResponse;
 import org.apache.hadoop.hive.metastore.api.Function;
+import org.apache.hadoop.hive.metastore.api.GetOpenTxnsInfoResponse;
 import org.apache.hadoop.hive.metastore.api.GetPrincipalsInRoleRequest;
 import org.apache.hadoop.hive.metastore.api.GetPrincipalsInRoleResponse;
 import org.apache.hadoop.hive.metastore.api.GetRoleGrantsForPrincipalRequest;
 import org.apache.hadoop.hive.metastore.api.GetRoleGrantsForPrincipalResponse;
+import org.apache.hadoop.hive.metastore.api.HeartbeatTxnRangeResponse;
 import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege;
 import org.apache.hadoop.hive.metastore.api.HiveObjectRef;
 import org.apache.hadoop.hive.metastore.api.Index;
@@ -70,8 +50,15 @@ import org.apache.hadoop.hive.metastore.api.InvalidInputException;
 import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
 import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
 import org.apache.hadoop.hive.metastore.api.InvalidPartitionException;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
 import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
+import org.apache.hadoop.hive.metastore.api.NotificationEvent;
+import org.apache.hadoop.hive.metastore.api.NotificationEventResponse;
+import org.apache.hadoop.hive.metastore.api.OpenTxnsResponse;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.PartitionEventType;
 import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet;
@@ -79,10 +66,20 @@ import org.apache.hadoop.hive.metastore.api.PrincipalType;
 import org.apache.hadoop.hive.metastore.api.PrivilegeBag;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.SetPartitionsStatsRequest;
+import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
+import org.apache.hadoop.hive.metastore.api.ShowLocksResponse;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
+import org.apache.hadoop.hive.metastore.api.TxnOpenException;
 import org.apache.hadoop.hive.metastore.api.UnknownDBException;
 import org.apache.hadoop.hive.metastore.api.UnknownPartitionException;
 import org.apache.hadoop.hive.metastore.api.UnknownTableException;
+import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
+import org.apache.thrift.TException;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
 
 /**
  * Wrapper around hive metastore thrift api
@@ -684,11 +681,11 @@ public interface IMetaStoreClient {
                         PartitionDropOptions options) throws TException;
 
   List<Partition> dropPartitions(String dbName, String tblName,
-      List<ObjectPair<Integer, byte[]>> partExprs, boolean deleteData, boolean ignoreProtection,
+      List<ObjectPair<Integer, byte[]>> partExprs, boolean deleteData,
       boolean ifExists) throws NoSuchObjectException, MetaException, TException;
 
   List<Partition> dropPartitions(String dbName, String tblName,
-      List<ObjectPair<Integer, byte[]>> partExprs, boolean deleteData, boolean ignoreProtection,
+      List<ObjectPair<Integer, byte[]>> partExprs, boolean deleteData,
       boolean ifExists, boolean needResults) throws NoSuchObjectException, MetaException, TException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
index 38dc406..907cbbf 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
@@ -1604,32 +1604,6 @@ public class MetaStoreUtils {
     return null;
   }
 
-  public static ProtectMode getProtectMode(Partition partition) {
-    return getProtectMode(partition.getParameters());
-  }
-
-  public static ProtectMode getProtectMode(Table table) {
-    return getProtectMode(table.getParameters());
-  }
-
-  private static ProtectMode getProtectMode(Map<String, String> parameters) {
-    if (parameters == null) {
-      return null;
-    }
-
-    if (!parameters.containsKey(ProtectMode.PARAMETER_NAME)) {
-      return new ProtectMode();
-    } else {
-      return ProtectMode.getProtectModeFromString(parameters.get(ProtectMode.PARAMETER_NAME));
-    }
-  }
-
-  public static boolean canDropPartition(Table table, Partition partition) {
-    ProtectMode mode = getProtectMode(partition);
-    ProtectMode parentMode = getProtectMode(table);
-    return (!mode.noDrop && !mode.offline && !mode.readOnly && !parentMode.noDropCascade);
-  }
-
   public static String ARCHIVING_LEVEL = "archiving_level";
   public static int getArchivingLevel(Partition part) throws MetaException {
     if (!isArchived(part)) {

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/metastore/src/java/org/apache/hadoop/hive/metastore/PartitionDropOptions.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/PartitionDropOptions.java b/metastore/src/java/org/apache/hadoop/hive/metastore/PartitionDropOptions.java
index 5b2811f..e8ffbd5 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/PartitionDropOptions.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/PartitionDropOptions.java
@@ -24,7 +24,6 @@ package org.apache.hadoop.hive.metastore;
 public class PartitionDropOptions {
 
   public boolean deleteData = true;
-  public boolean ignoreProtection = false;
   public boolean ifExists = false;
   public boolean returnResults = true;
   public boolean purgeData = false;
@@ -36,11 +35,6 @@ public class PartitionDropOptions {
     return this;
   }
 
-  public PartitionDropOptions ignoreProtection(boolean ignoreProtection) {
-    this.ignoreProtection = ignoreProtection;
-    return this;
-  }
-
   public PartitionDropOptions ifExists(boolean ifExists) {
     this.ifExists = ifExists;
     return this;

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/metastore/src/java/org/apache/hadoop/hive/metastore/ProtectMode.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/ProtectMode.java b/metastore/src/java/org/apache/hadoop/hive/metastore/ProtectMode.java
deleted file mode 100644
index b8f1390..0000000
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/ProtectMode.java
+++ /dev/null
@@ -1,97 +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.metastore;
-
-
-public class ProtectMode {
-  public static String PARAMETER_NAME = "PROTECT_MODE";
-
-  public static String FLAG_OFFLINE = "OFFLINE";
-  public static String FLAG_NO_DROP = "NO_DROP";
-  public static String FLAG_NO_DROP_CASCADE = "NO_DROP_CASCADE";
-  public static String FLAG_READ_ONLY = "READ_ONLY";
-
-  public boolean offline = false;
-  public boolean readOnly = false;
-  public boolean noDrop = false;
-  public boolean noDropCascade = false;
-
-  static public ProtectMode getProtectModeFromString(String sourceString) {
-    return new ProtectMode(sourceString);
-  }
-
-  private ProtectMode(String sourceString) {
-    String[] tokens = sourceString.split(",");
-    for (String token: tokens) {
-      if (token.equalsIgnoreCase(FLAG_OFFLINE)) {
-        offline = true;
-      } else if (token.equalsIgnoreCase(FLAG_NO_DROP)) {
-        noDrop = true;
-      } else if (token.equalsIgnoreCase(FLAG_NO_DROP_CASCADE)) {
-        noDropCascade = true;
-      } else if (token.equalsIgnoreCase(FLAG_READ_ONLY)) {
-        readOnly = true;
-      }
-    }
-  }
-
-  public ProtectMode() {
-  }
-
-  @Override
-  public String toString() {
-    String retString = null;
-
-    if (offline) {
-        retString = FLAG_OFFLINE;
-    }
-
-    if (noDrop) {
-      if (retString != null) {
-        retString = retString + "," + FLAG_NO_DROP;
-      }
-      else
-      {
-        retString = FLAG_NO_DROP;
-      }
-    }
-
-    if (noDropCascade) {
-      if (retString != null) {
-        retString = retString + "," + FLAG_NO_DROP_CASCADE;
-      }
-      else
-      {
-        retString = FLAG_NO_DROP_CASCADE;
-      }
-    }
-
-    if (readOnly) {
-      if (retString != null) {
-        retString = retString + "," + FLAG_READ_ONLY;
-      }
-      else
-      {
-        retString = FLAG_READ_ONLY;
-      }
-    }
-
-    return retString;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
index a8c6aca..734742c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
@@ -65,7 +65,6 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.PartitionDropOptions;
-import org.apache.hadoop.hive.metastore.ProtectMode;
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
@@ -3210,17 +3209,6 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     return builder;
   }
 
-  private void setAlterProtectMode(boolean protectModeEnable,
-      AlterTableDesc.ProtectModeType protectMode,
-      ProtectMode mode) {
-    if (protectMode == AlterTableDesc.ProtectModeType.OFFLINE) {
-      mode.offline = protectModeEnable;
-    } else if (protectMode == AlterTableDesc.ProtectModeType.NO_DROP) {
-      mode.noDrop = protectModeEnable;
-    } else if (protectMode == AlterTableDesc.ProtectModeType.NO_DROP_CASCADE) {
-      mode.noDropCascade = protectModeEnable;
-    }
-  }
   /**
    * Alter a given table.
    *
@@ -3453,20 +3441,6 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       if (alterTbl.getSerdeName() != null) {
         sd.getSerdeInfo().setSerializationLib(alterTbl.getSerdeName());
       }
-    } else if (alterTbl.getOp() == AlterTableDesc.AlterTableTypes.ALTERPROTECTMODE) {
-      boolean protectModeEnable = alterTbl.isProtectModeEnable();
-      AlterTableDesc.ProtectModeType protectMode = alterTbl.getProtectModeType();
-
-      ProtectMode mode = null;
-      if (part != null) {
-        mode = part.getProtectMode();
-        setAlterProtectMode(protectModeEnable, protectMode, mode);
-        part.setProtectMode(mode);
-      } else {
-        mode = tbl.getProtectMode();
-        setAlterProtectMode(protectModeEnable,protectMode, mode);
-        tbl.setProtectMode(mode);
-      }
     } else if (alterTbl.getOp() == AlterTableDesc.AlterTableTypes.ADDCLUSTERSORTCOLUMN) {
       StorageDescriptor sd = (part == null ? tbl.getTTable().getSd() : part.getTPartition().getSd());
       // validate sort columns and bucket columns
@@ -3635,7 +3609,6 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
                             dropTbl.getPartSpecs(),
                             PartitionDropOptions.instance()
                                                 .deleteData(true)
-                                                .ignoreProtection(dropTbl.getIgnoreProtection())
                                                 .ifExists(true)
                                                 .purgeData(dropTbl.getIfPurge()));
     for (Partition partition : droppedParts) {
@@ -3666,11 +3639,6 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       }
     }
 
-    if (tbl != null && !tbl.canDrop()) {
-      throw new HiveException("Table " + tbl.getTableName() +
-          " is protected from being dropped");
-    }
-
     ReplicationSpec replicationSpec = dropTbl.getReplicationSpec();
     if ((tbl!= null) && replicationSpec.isInReplicationScope()){
       /**
@@ -3714,24 +3682,6 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     int partitionBatchSize = HiveConf.getIntVar(conf,
         ConfVars.METASTORE_BATCH_RETRIEVE_TABLE_PARTITION_MAX);
 
-    // We should check that all the partitions of the table can be dropped
-    if (tbl != null && tbl.isPartitioned()) {
-      List<String> partitionNames = db.getPartitionNames(tbl.getDbName(), tbl.getTableName(), (short)-1);
-
-      for(int i=0; i < partitionNames.size(); i+= partitionBatchSize) {
-        List<String> partNames = partitionNames.subList(i, Math.min(i+partitionBatchSize,
-            partitionNames.size()));
-        List<Partition> listPartitions = db.getPartitionsByNames(tbl, partNames);
-        for (Partition p: listPartitions) {
-          if (!p.canDrop()) {
-            throw new HiveException("Table " + tbl.getTableName() +
-                " Partition" + p.getName() +
-                " is protected from being dropped");
-          }
-        }
-      }
-    }
-
     // drop the table
     db.dropTable(dropTbl.getTableName(), dropTbl.getIfPurge());
     if (tbl != null) {

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/java/org/apache/hadoop/hive/ql/hooks/WriteEntity.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/WriteEntity.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/WriteEntity.java
index 968c1e1..298e7f0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/hooks/WriteEntity.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/WriteEntity.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.hive.ql.hooks;
 
-import java.io.Serializable;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.Path;
@@ -29,6 +27,8 @@ import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.plan.AlterTableDesc;
 
+import java.io.Serializable;
+
 /**
  * This class encapsulates an object that is being written to by the query. This
  * object may be a table, partition, dfs directory or a local directory.
@@ -193,8 +193,6 @@ public class WriteEntity extends Entity implements Serializable {
       case REPLACECOLS:
       case ARCHIVE:
       case UNARCHIVE:
-      case ALTERPROTECTMODE:
-      case ALTERPARTITIONPROTECTMODE:
       case ALTERLOCATION:
       case DROPPARTITION:
       case RENAMEPARTITION:

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
index d89aafc..00125fa 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
@@ -18,30 +18,7 @@
 
 package org.apache.hadoop.hive.ql.metadata;
 
-import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_STORAGE;
-import static org.apache.hadoop.hive.serde.serdeConstants.COLLECTION_DELIM;
-import static org.apache.hadoop.hive.serde.serdeConstants.ESCAPE_CHAR;
-import static org.apache.hadoop.hive.serde.serdeConstants.FIELD_DELIM;
-import static org.apache.hadoop.hive.serde.serdeConstants.LINE_DELIM;
-import static org.apache.hadoop.hive.serde.serdeConstants.MAPKEY_DELIM;
-import static org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_FORMAT;
-import static org.apache.hadoop.hive.serde.serdeConstants.STRING_TYPE_NAME;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.LinkedHashSet;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
+import com.google.common.collect.Sets;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -122,7 +99,29 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.thrift.TException;
 
-import com.google.common.collect.Sets;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_STORAGE;
+import static org.apache.hadoop.hive.serde.serdeConstants.COLLECTION_DELIM;
+import static org.apache.hadoop.hive.serde.serdeConstants.ESCAPE_CHAR;
+import static org.apache.hadoop.hive.serde.serdeConstants.FIELD_DELIM;
+import static org.apache.hadoop.hive.serde.serdeConstants.LINE_DELIM;
+import static org.apache.hadoop.hive.serde.serdeConstants.MAPKEY_DELIM;
+import static org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_FORMAT;
+import static org.apache.hadoop.hive.serde.serdeConstants.STRING_TYPE_NAME;
 
 
 /**
@@ -1979,19 +1978,17 @@ private void constructOneLBLocationMap(FileStatus fSta,
   }
 
   public List<Partition> dropPartitions(String tblName, List<DropTableDesc.PartSpec> partSpecs,
-      boolean deleteData, boolean ignoreProtection, boolean ifExists) throws HiveException {
+      boolean deleteData, boolean ifExists) throws HiveException {
     String[] names = Utilities.getDbTableName(tblName);
-    return dropPartitions(
-        names[0], names[1], partSpecs, deleteData, ignoreProtection, ifExists);
+    return dropPartitions(names[0], names[1], partSpecs, deleteData, ifExists);
   }
 
   public List<Partition> dropPartitions(String dbName, String tblName,
-      List<DropTableDesc.PartSpec> partSpecs,  boolean deleteData, boolean ignoreProtection,
+      List<DropTableDesc.PartSpec> partSpecs,  boolean deleteData,
       boolean ifExists) throws HiveException {
     return dropPartitions(dbName, tblName, partSpecs,
                           PartitionDropOptions.instance()
                                               .deleteData(deleteData)
-                                              .ignoreProtection(ignoreProtection)
                                               .ifExists(ifExists));
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/java/org/apache/hadoop/hive/ql/metadata/Partition.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Partition.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Partition.java
index 08ff2e9..2e77bc4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Partition.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Partition.java
@@ -34,7 +34,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
-import org.apache.hadoop.hive.metastore.ProtectMode;
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.MetaException;
@@ -541,56 +540,6 @@ public class Partition implements Serializable {
   }
 
   /**
-   * @param protectMode
-   */
-  public void setProtectMode(ProtectMode protectMode){
-    Map<String, String> parameters = tPartition.getParameters();
-    String pm = protectMode.toString();
-    if (pm != null) {
-      parameters.put(ProtectMode.PARAMETER_NAME, pm);
-    } else {
-      parameters.remove(ProtectMode.PARAMETER_NAME);
-    }
-    tPartition.setParameters(parameters);
-  }
-
-  /**
-   * @return protect mode
-   */
-  public ProtectMode getProtectMode(){
-    return MetaStoreUtils.getProtectMode(tPartition);
-  }
-
-  /**
-   * @return True protect mode indicates the partition if offline.
-   */
-  public boolean isOffline(){
-    ProtectMode pm = getProtectMode();
-    if (pm == null) {
-      return false;
-    } else {
-      return pm.offline;
-    }
-  }
-
-  /**
-   * @return True if protect mode attribute of the partition indicate
-   * that it is OK to drop the table
-   */
-  public boolean canDrop() {
-    return MetaStoreUtils.canDropPartition(table.getTTable(), tPartition);
-  }
-
-  /**
-   * @return True if protect mode attribute of the partition indicate
-   * that it is OK to write to the table
-   */
-  public boolean canWrite() {
-    ProtectMode mode = getProtectMode();
-    return (!mode.offline && !mode.readOnly);
-  }
-
-  /**
    * @return include the db name
    */
   public String getCompleteName() {

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
index e53933e..52ed4a3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
@@ -18,15 +18,6 @@
 
 package org.apache.hadoop.hive.ql.metadata;
 
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-
 import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -37,7 +28,6 @@ import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
-import org.apache.hadoop.hive.metastore.ProtectMode;
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.MetaException;
@@ -65,6 +55,15 @@ import org.apache.hadoop.mapred.InputFormat;
 import org.apache.hadoop.mapred.OutputFormat;
 import org.apache.hadoop.mapred.SequenceFileInputFormat;
 
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
 /**
  * A Hive Table: is a fundamental unit of data in Hive that shares a common schema/DDL.
  *
@@ -849,52 +848,6 @@ public class Table implements Serializable {
   }
 
   /**
-   * @param protectMode
-   */
-  public void setProtectMode(ProtectMode protectMode){
-    Map<String, String> parameters = tTable.getParameters();
-    String pm = protectMode.toString();
-    if (pm != null) {
-      parameters.put(ProtectMode.PARAMETER_NAME, pm);
-    } else {
-      parameters.remove(ProtectMode.PARAMETER_NAME);
-    }
-    tTable.setParameters(parameters);
-  }
-
-  /**
-   * @return protect mode
-   */
-  public ProtectMode getProtectMode(){
-    return MetaStoreUtils.getProtectMode(tTable);
-  }
-
-  /**
-   * @return True protect mode indicates the table if offline.
-   */
-  public boolean isOffline(){
-    return getProtectMode().offline;
-  }
-
-  /**
-   * @return True if protect mode attribute of the partition indicate
-   * that it is OK to drop the partition
-   */
-  public boolean canDrop() {
-    ProtectMode mode = getProtectMode();
-    return (!mode.noDrop && !mode.offline && !mode.readOnly && !mode.noDropCascade);
-  }
-
-  /**
-   * @return True if protect mode attribute of the table indicate
-   * that it is OK to write the table
-   */
-  public boolean canWrite() {
-    ProtectMode mode = getProtectMode();
-    return (!mode.offline && !mode.readOnly);
-  }
-
-  /**
    * @return include the db name
    */
   public String getCompleteName() {

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java
index bc09fc3..a78700d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java
@@ -18,16 +18,6 @@
 
 package org.apache.hadoop.hive.ql.metadata.formatting;
 
-import java.math.BigInteger;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
 import org.apache.commons.lang.StringEscapeUtils;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -55,6 +45,16 @@ import org.apache.hadoop.hive.ql.plan.PlanUtils;
 import org.apache.hadoop.hive.ql.plan.ShowIndexesDesc;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
 
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
 
 /**
  * This class provides methods to format table and index information.
@@ -357,8 +357,6 @@ public final class MetaDataFormatUtils {
     formatOutput("Owner:", tbl.getOwner(), tableInfo);
     formatOutput("CreateTime:", formatDate(tbl.getTTable().getCreateTime()), tableInfo);
     formatOutput("LastAccessTime:", formatDate(tbl.getTTable().getLastAccessTime()), tableInfo);
-    String protectMode = tbl.getProtectMode().toString();
-    formatOutput("Protect Mode:", protectMode == null ? "None" : protectMode, tableInfo);
     formatOutput("Retention:", Integer.toString(tbl.getRetention()), tableInfo);
     if (!tbl.isView()) {
       formatOutput("Location:", tbl.getDataLocation().toString(), tableInfo);
@@ -378,8 +376,6 @@ public final class MetaDataFormatUtils {
     formatOutput("CreateTime:", formatDate(part.getTPartition().getCreateTime()), tableInfo);
     formatOutput("LastAccessTime:", formatDate(part.getTPartition().getLastAccessTime()),
         tableInfo);
-    String protectMode = part.getProtectMode().toString();
-    formatOutput("Protect Mode:", protectMode == null ? "None" : protectMode, tableInfo);
     formatOutput("Location:", part.getLocation(), tableInfo);
 
     if (part.getTPartition().getParameters().size() > 0) {

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
index 24ca663..21625bc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
@@ -18,28 +18,7 @@
 
 package org.apache.hadoop.hive.ql.parse;
 
-import static org.apache.hadoop.hive.ql.parse.HiveParser.TOK_DATABASELOCATION;
-import static org.apache.hadoop.hive.ql.parse.HiveParser.TOK_DATABASEPROPERTIES;
-
-import java.io.Serializable;
-import java.lang.reflect.Constructor;
-import java.lang.reflect.InvocationTargetException;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Properties;
-import java.util.Set;
-
+import com.google.common.collect.Lists;
 import org.antlr.runtime.tree.CommonTree;
 import org.antlr.runtime.tree.Tree;
 import org.apache.commons.logging.Log;
@@ -158,7 +137,27 @@ import org.apache.hadoop.mapred.InputFormat;
 import org.apache.hadoop.mapred.TextInputFormat;
 import org.apache.hadoop.util.StringUtils;
 
-import com.google.common.collect.Lists;
+import java.io.Serializable;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Properties;
+import java.util.Set;
+
+import static org.apache.hadoop.hive.ql.parse.HiveParser.TOK_DATABASELOCATION;
+import static org.apache.hadoop.hive.ql.parse.HiveParser.TOK_DATABASEPROPERTIES;
 
 /**
  * DDLSemanticAnalyzer.
@@ -288,8 +287,6 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
         analyzeExchangePartition(qualified, ast);
       } else if (ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_FILEFORMAT) {
         analyzeAlterTableFileFormat(ast, tableName, partSpec);
-      } else if (ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_PROTECTMODE) {
-        analyzeAlterTableProtectMode(ast, tableName, partSpec);
       } else if (ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_LOCATION) {
         analyzeAlterTableLocation(ast, tableName, partSpec);
       } else if (ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_MERGEFILES) {
@@ -1476,56 +1473,6 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
 
   }
 
-  private void analyzeAlterTableProtectMode(ASTNode ast, String tableName,
-      HashMap<String, String> partSpec)
-      throws SemanticException {
-
-    AlterTableDesc alterTblDesc =
-        new AlterTableDesc(AlterTableTypes.ALTERPROTECTMODE);
-
-    alterTblDesc.setOldName(tableName);
-    alterTblDesc.setPartSpec(partSpec);
-
-    ASTNode child = (ASTNode) ast.getChild(0);
-
-    switch (child.getToken().getType()) {
-    case HiveParser.TOK_ENABLE:
-      alterTblDesc.setProtectModeEnable(true);
-      break;
-    case HiveParser.TOK_DISABLE:
-      alterTblDesc.setProtectModeEnable(false);
-      break;
-    default:
-      throw new SemanticException(
-          "Set Protect mode Syntax parsing error.");
-    }
-
-    ASTNode grandChild = (ASTNode) child.getChild(0);
-    switch (grandChild.getToken().getType()) {
-    case HiveParser.TOK_OFFLINE:
-      alterTblDesc.setProtectModeType(AlterTableDesc.ProtectModeType.OFFLINE);
-      break;
-    case HiveParser.TOK_NO_DROP:
-      if (grandChild.getChildCount() > 0) {
-        alterTblDesc.setProtectModeType(AlterTableDesc.ProtectModeType.NO_DROP_CASCADE);
-      }
-      else {
-        alterTblDesc.setProtectModeType(AlterTableDesc.ProtectModeType.NO_DROP);
-      }
-      break;
-    case HiveParser.TOK_READONLY:
-      throw new SemanticException(
-          "Potect mode READONLY is not implemented");
-    default:
-      throw new SemanticException(
-          "Only protect mode NO_DROP or OFFLINE supported");
-    }
-
-    addInputsOutputsAlterTable(tableName, partSpec, alterTblDesc);
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        alterTblDesc), conf));
-  }
-
   private void analyzeAlterTablePartMergeFiles(ASTNode ast,
       String tableName, HashMap<String, String> partSpec)
       throws SemanticException {
@@ -2690,11 +2637,10 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     re.noLockNeeded();
     inputs.add(re);
 
-    boolean ignoreProtection = ast.getFirstChildWithType(HiveParser.TOK_IGNOREPROTECTION) != null;
-    addTableDropPartsOutputs(tab, partSpecs.values(), !ifExists, ignoreProtection);
+    addTableDropPartsOutputs(tab, partSpecs.values(), !ifExists);
 
     DropTableDesc dropTblDesc =
-        new DropTableDesc(getDotName(qualified), partSpecs, expectView, ignoreProtection, mustPurge, replicationSpec);
+        new DropTableDesc(getDotName(qualified), partSpecs, expectView, mustPurge, replicationSpec);
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), dropTblDesc), conf));
   }
 
@@ -3165,9 +3111,8 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
    * throwIfNonExistent is true, otherwise ignore it.
    */
   private void addTableDropPartsOutputs(Table tab,
-      Collection<List<ExprNodeGenericFuncDesc>> partSpecs, boolean throwIfNonExistent,
-      boolean ignoreProtection) throws SemanticException {
-
+                                        Collection<List<ExprNodeGenericFuncDesc>> partSpecs,
+                                        boolean throwIfNonExistent) throws SemanticException {
     for (List<ExprNodeGenericFuncDesc> specs : partSpecs) {
       for (ExprNodeGenericFuncDesc partSpec : specs) {
         List<Partition> parts = new ArrayList<Partition>();
@@ -3193,11 +3138,6 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
           }
         }
         for (Partition p : parts) {
-          // TODO: same thing, metastore already checks this but check here if we can.
-          if (!ignoreProtection && !p.canDrop()) {
-            throw new SemanticException(
-              ErrorMsg.DROP_COMMAND_NOT_ALLOWED_FOR_PARTITION.getMsg(p.getCompleteName()));
-          }
           outputs.add(new WriteEntity(p, WriteEntity.WriteType.DDL_EXCLUSIVE));
         }
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/java/org/apache/hadoop/hive/ql/parse/EximUtil.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/EximUtil.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/EximUtil.java
index a4c5d0e..bdf0ed7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/EximUtil.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/EximUtil.java
@@ -18,20 +18,6 @@
 
 package org.apache.hadoop.hive.ql.parse;
 
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.util.AbstractMap;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.StringTokenizer;
-import java.util.TreeMap;
-
 import com.google.common.base.Function;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
@@ -56,6 +42,18 @@ import org.json.JSONException;
 import org.json.JSONObject;
 
 import javax.annotation.Nullable;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.StringTokenizer;
+import java.util.TreeMap;
 
 /**
  *
@@ -129,11 +127,6 @@ public class EximUtil {
   }
 
   static void validateTable(org.apache.hadoop.hive.ql.metadata.Table table) throws SemanticException {
-    if (table.isOffline()) {
-      throw new SemanticException(
-          ErrorMsg.OFFLINE_TABLE_OR_PARTITION.getMsg(":Table "
-              + table.getTableName()));
-    }
     if (table.isView()) {
       throw new SemanticException(ErrorMsg.DML_AGAINST_VIEW.getMsg());
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/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 bdd7cb7..85c0ae6 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
@@ -110,8 +110,6 @@ KW_AFTER: 'AFTER';
 KW_DESCRIBE: 'DESCRIBE';
 KW_DROP: 'DROP';
 KW_RENAME: 'RENAME';
-KW_IGNORE: 'IGNORE';
-KW_PROTECTION: 'PROTECTION';
 KW_TO: 'TO';
 KW_COMMENT: 'COMMENT';
 KW_BOOLEAN: 'BOOLEAN';
@@ -157,11 +155,8 @@ KW_INPUTFORMAT: 'INPUTFORMAT';
 KW_OUTPUTFORMAT: 'OUTPUTFORMAT';
 KW_INPUTDRIVER: 'INPUTDRIVER';
 KW_OUTPUTDRIVER: 'OUTPUTDRIVER';
-KW_OFFLINE: 'OFFLINE';
 KW_ENABLE: 'ENABLE';
 KW_DISABLE: 'DISABLE';
-KW_READONLY: 'READONLY';
-KW_NO_DROP: 'NO_DROP';
 KW_LOCATION: 'LOCATION';
 KW_TABLESAMPLE: 'TABLESAMPLE';
 KW_BUCKET: 'BUCKET';

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/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 15f1f11..3f95bb8 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
@@ -151,7 +151,6 @@ TOK_ALTERTABLE_REPLACECOLS;
 TOK_ALTERTABLE_ADDPARTS;
 TOK_ALTERTABLE_DROPPARTS;
 TOK_ALTERTABLE_PARTCOLTYPE;
-TOK_ALTERTABLE_PROTECTMODE;
 TOK_ALTERTABLE_MERGEFILES;
 TOK_ALTERTABLE_TOUCH;
 TOK_ALTERTABLE_ARCHIVE;
@@ -330,7 +329,6 @@ TOK_WINDOWDEF;
 TOK_WINDOWSPEC;
 TOK_WINDOWVALUES;
 TOK_WINDOWRANGE;
-TOK_IGNOREPROTECTION;
 TOK_SUBQUERY_EXPR;
 TOK_SUBQUERY_OP;
 TOK_SUBQUERY_OP_NOTIN;
@@ -809,13 +807,6 @@ orReplace
     -> ^(TOK_ORREPLACE)
     ;
 
-ignoreProtection
-@init { pushMsg("ignore protection clause", state); }
-@after { popMsg(state); }
-        : KW_IGNORE KW_PROTECTION
-        -> ^(TOK_IGNOREPROTECTION)
-        ;
-
 createDatabaseStatement
 @init { pushMsg("create database statement", state); }
 @after { popMsg(state); }
@@ -1022,7 +1013,6 @@ alterTblPartitionStatementSuffix
 @after {popMsg(state);}
   : alterStatementSuffixFileFormat
   | alterStatementSuffixLocation
-  | alterStatementSuffixProtectMode
   | alterStatementSuffixMergeFiles
   | alterStatementSuffixSerdeProperties
   | alterStatementSuffixRenamePart
@@ -1166,9 +1156,9 @@ partitionLocation
 alterStatementSuffixDropPartitions[boolean table]
 @init { pushMsg("drop partition statement", state); }
 @after { popMsg(state); }
-    : KW_DROP ifExists? dropPartitionSpec (COMMA dropPartitionSpec)* ignoreProtection? KW_PURGE? replicationClause?
-    -> { table }? ^(TOK_ALTERTABLE_DROPPARTS dropPartitionSpec+ ifExists? ignoreProtection? KW_PURGE? replicationClause?)
-    ->            ^(TOK_ALTERVIEW_DROPPARTS dropPartitionSpec+ ifExists? ignoreProtection? replicationClause?)
+    : KW_DROP ifExists? dropPartitionSpec (COMMA dropPartitionSpec)* KW_PURGE? replicationClause?
+    -> { table }? ^(TOK_ALTERTABLE_DROPPARTS dropPartitionSpec+ ifExists? KW_PURGE? replicationClause?)
+    ->            ^(TOK_ALTERVIEW_DROPPARTS dropPartitionSpec+ ifExists? replicationClause?)
     ;
 
 alterStatementSuffixProperties
@@ -1276,13 +1266,6 @@ alterStatementSuffixExchangePartition
     -> ^(TOK_ALTERTABLE_EXCHANGEPARTITION partitionSpec $exchangename)
     ;
 
-alterStatementSuffixProtectMode
-@init { pushMsg("alter partition protect mode statement", state); }
-@after { popMsg(state); }
-    : alterProtectMode
-    -> ^(TOK_ALTERTABLE_PROTECTMODE alterProtectMode)
-    ;
-
 alterStatementSuffixRenamePart
 @init { pushMsg("alter table rename partition statement", state); }
 @after { popMsg(state); }
@@ -1304,21 +1287,6 @@ alterStatementSuffixMergeFiles
     -> ^(TOK_ALTERTABLE_MERGEFILES)
     ;
 
-alterProtectMode
-@init { pushMsg("protect mode specification enable", state); }
-@after { popMsg(state); }
-    : KW_ENABLE alterProtectModeMode  -> ^(TOK_ENABLE alterProtectModeMode)
-    | KW_DISABLE alterProtectModeMode  -> ^(TOK_DISABLE alterProtectModeMode)
-    ;
-
-alterProtectModeMode
-@init { pushMsg("protect mode specification enable", state); }
-@after { popMsg(state); }
-    : KW_OFFLINE  -> ^(TOK_OFFLINE)
-    | KW_NO_DROP KW_CASCADE? -> ^(TOK_NO_DROP KW_CASCADE?)
-    | KW_READONLY  -> ^(TOK_READONLY)
-    ;
-
 alterStatementSuffixBucketNum
 @init { pushMsg("", state); }
 @after { popMsg(state); }

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
index 944cee4..85fa9c9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java
@@ -206,12 +206,7 @@ public class LoadSemanticAnalyzer extends BaseSemanticAnalyzer {
     // initialize destination table/partition
     TableSpec ts = new TableSpec(db, conf, (ASTNode) tableTree);
 
-    if (ts.tableHandle.isOffline()){
-      throw new SemanticException(
-          ErrorMsg.OFFLINE_TABLE_OR_PARTITION.getMsg(":Table " + ts.tableName));
-    }
-
-    if (ts.tableHandle.isView()) {
+   if (ts.tableHandle.isView()) {
       throw new SemanticException(ErrorMsg.DML_AGAINST_VIEW.getMsg());
     }
     if (ts.tableHandle.isNonNative()) {
@@ -255,10 +250,6 @@ public class LoadSemanticAnalyzer extends BaseSemanticAnalyzer {
       try{
         Partition part = Hive.get().getPartition(ts.tableHandle, partSpec, false);
         if (part != null) {
-          if (part.isOffline()) {
-            throw new SemanticException(ErrorMsg.OFFLINE_TABLE_OR_PARTITION.
-                getMsg(ts.tableName + ":" + part.getName()));
-          }
           if (isOverWrite){
             outputs.add(new WriteEntity(part, WriteEntity.WriteType.INSERT_OVERWRITE));
           } else {

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/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 8516631..aab4250 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
@@ -1601,19 +1601,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
           throw new SemanticException(ErrorMsg.ACID_OP_ON_NONACID_TABLE, tab_name);
         }
 
-        // We check offline of the table, as if people only select from an
-        // non-existing partition of an offline table, the partition won't
-        // be added to inputs and validate() won't have the information to
-        // check the table's offline status.
-        // TODO: Modify the code to remove the checking here and consolidate
-        // it in validate()
-        //
-        if (tab.isOffline()) {
-          throw new SemanticException(ErrorMsg.OFFLINE_TABLE_OR_PARTITION.
-              getMsg("Table " + getUnescapedName(qb.getParseInfo().getSrcForAlias(alias))));
-        }
-
-        if (tab.isView()) {
+       if (tab.isView()) {
           if (qb.getParseInfo().isAnalyzeCommand()) {
             throw new SemanticException(ErrorMsg.ANALYZE_VIEW.getMsg());
           }
@@ -10569,20 +10557,6 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
 
       Table tbl = readEntity.getTable();
       Partition p = readEntity.getPartition();
-
-
-      if (tbl.isOffline()) {
-        throw new SemanticException(
-            ErrorMsg.OFFLINE_TABLE_OR_PARTITION.getMsg(
-                "Table " + tbl.getTableName()));
-      }
-
-      if (type == ReadEntity.Type.PARTITION && p != null && p.isOffline()) {
-        throw new SemanticException(
-            ErrorMsg.OFFLINE_TABLE_OR_PARTITION.getMsg(
-                "Table " + tbl.getTableName() +
-                    " Partition " + p.getName()));
-      }
     }
 
     for (WriteEntity writeEntity : getOutputs()) {
@@ -10636,25 +10610,11 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         } catch (HiveException e) {
           throw new SemanticException(e);
         }
-
-        if (type == WriteEntity.Type.PARTITION && p != null && p.isOffline()) {
-          throw new SemanticException(
-              ErrorMsg.OFFLINE_TABLE_OR_PARTITION.getMsg(
-                  " Table " + tbl.getTableName() +
-                      " Partition " + p.getName()));
-        }
-
       }
       else {
         LOG.debug("Not a partition.");
         tbl = writeEntity.getTable();
       }
-
-      if (tbl.isOffline()) {
-        throw new SemanticException(
-            ErrorMsg.OFFLINE_TABLE_OR_PARTITION.getMsg(
-                "Table " + tbl.getTableName()));
-      }
     }
 
     boolean reworkMapredWork = HiveConf.getBoolVar(this.conf,

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/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 97d02ea..2fdf1e7 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
@@ -18,13 +18,13 @@
 
 package org.apache.hadoop.hive.ql.parse;
 
-import java.util.HashMap;
-
 import org.antlr.runtime.tree.Tree;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.plan.HiveOperation;
 import org.apache.hadoop.hive.ql.session.SessionState;
 
+import java.util.HashMap;
+
 /**
  * SemanticAnalyzerFactory.
  *
@@ -114,10 +114,6 @@ public final class SemanticAnalyzerFactory {
   }
 
   static {
-    tablePartitionCommandType.put(
-        HiveParser.TOK_ALTERTABLE_PROTECTMODE,
-        new HiveOperation[] { HiveOperation.ALTERTABLE_PROTECTMODE,
-            HiveOperation.ALTERPARTITION_PROTECTMODE });
     tablePartitionCommandType.put(HiveParser.TOK_ALTERTABLE_FILEFORMAT,
         new HiveOperation[] { HiveOperation.ALTERTABLE_FILEFORMAT,
             HiveOperation.ALTERPARTITION_FILEFORMAT });


[20/50] [abbrv] hive git commit: HIVE-11136 - Unused Logger in org.apache.hadoop.hive.ql.udf.generic.GenericUDAFCount (Yongzhi Chen, reviewed by Chao Sun)

Posted by ga...@apache.org.
HIVE-11136 - Unused Logger in org.apache.hadoop.hive.ql.udf.generic.GenericUDAFCount (Yongzhi Chen, 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/e61a1a94
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/e61a1a94
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/e61a1a94

Branch: refs/heads/hbase-metastore
Commit: e61a1a9412ea15da72fb7645112f7f7572688ece
Parents: d6ec52e
Author: Yongzhi Chen <yo...@hotmail.com>
Authored: Wed Jul 15 18:27:55 2015 -0700
Committer: Chao Sun <su...@apache.org>
Committed: Wed Jul 15 18:27:55 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hive/ql/udf/generic/GenericUDAFCollectList.java      | 5 -----
 .../hadoop/hive/ql/udf/generic/GenericUDAFCollectSet.java       | 5 -----
 .../org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCount.java | 4 ----
 .../hadoop/hive/ql/udf/generic/GenericUDAFCovariance.java       | 4 ----
 .../apache/hadoop/hive/ql/udf/generic/GenericUDAFCumeDist.java  | 4 ----
 .../apache/hadoop/hive/ql/udf/generic/GenericUDAFDenseRank.java | 4 ----
 6 files changed, 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/e61a1a94/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCollectList.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCollectList.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCollectList.java
index b10c4ab..156d19b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCollectList.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCollectList.java
@@ -18,20 +18,15 @@
 
 package org.apache.hadoop.hive.ql.udf.generic;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.ql.exec.Description;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFMkCollectionEvaluator.BufferType;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 
 @Description(name = "collect_list", value = "_FUNC_(x) - Returns a list of objects with duplicates")
 public class GenericUDAFCollectList extends AbstractGenericUDAFResolver {
 
-  static final Log LOG = LogFactory.getLog(GenericUDAFCollectList.class.getName());
-
   public GenericUDAFCollectList() {
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/e61a1a94/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCollectSet.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCollectSet.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCollectSet.java
index 312a698..0c2cf90 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCollectSet.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCollectSet.java
@@ -17,13 +17,10 @@
  */
 package org.apache.hadoop.hive.ql.udf.generic;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.ql.exec.Description;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFMkCollectionEvaluator.BufferType;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 
 /**
@@ -32,8 +29,6 @@ import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 @Description(name = "collect_set", value = "_FUNC_(x) - Returns a set of objects with duplicate elements eliminated")
 public class GenericUDAFCollectSet extends AbstractGenericUDAFResolver {
 
-  static final Log LOG = LogFactory.getLog(GenericUDAFCollectSet.class.getName());
-
   public GenericUDAFCollectSet() {
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/e61a1a94/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCount.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCount.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCount.java
index d47e7f9..eaf112e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCount.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCount.java
@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.hive.ql.udf.generic;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.ql.exec.Description;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -44,8 +42,6 @@ import org.apache.hadoop.io.LongWritable;
           +        "which the supplied expression(s) are unique and non-NULL.")
 public class GenericUDAFCount implements GenericUDAFResolver2 {
 
-  private static final Log LOG = LogFactory.getLog(GenericUDAFCount.class.getName());
-
   @Override
   public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters)
       throws SemanticException {

http://git-wip-us.apache.org/repos/asf/hive/blob/e61a1a94/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCovariance.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCovariance.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCovariance.java
index f1017be..8221c1b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCovariance.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCovariance.java
@@ -19,8 +19,6 @@ package org.apache.hadoop.hive.ql.udf.generic;
 
 import java.util.ArrayList;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.ql.exec.Description;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -64,8 +62,6 @@ import org.apache.hadoop.io.LongWritable;
         + "where neither x nor y is null.")
 public class GenericUDAFCovariance extends AbstractGenericUDAFResolver {
 
-  static final Log LOG = LogFactory.getLog(GenericUDAFCovariance.class.getName());
-
   @Override
   public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters) throws SemanticException {
     if (parameters.length != 2) {

http://git-wip-us.apache.org/repos/asf/hive/blob/e61a1a94/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCumeDist.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCumeDist.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCumeDist.java
index bc93204..611966c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCumeDist.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCumeDist.java
@@ -21,8 +21,6 @@ package org.apache.hadoop.hive.ql.udf.generic;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.ql.exec.Description;
 import org.apache.hadoop.hive.ql.exec.WindowFunctionDescription;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -49,8 +47,6 @@ import org.apache.hadoop.io.IntWritable;
 )
 public class GenericUDAFCumeDist extends GenericUDAFRank {
 
-  static final Log LOG = LogFactory.getLog(GenericUDAFCumeDist.class.getName());
-
   @Override
   protected GenericUDAFAbstractRankEvaluator createEvaluator() {
     return new GenericUDAFCumeDistEvaluator();

http://git-wip-us.apache.org/repos/asf/hive/blob/e61a1a94/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFDenseRank.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFDenseRank.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFDenseRank.java
index 50ee4ef..715454b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFDenseRank.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFDenseRank.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.hive.ql.udf.generic;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.ql.exec.Description;
 import org.apache.hadoop.hive.ql.exec.WindowFunctionDescription;
 
@@ -39,8 +37,6 @@ import org.apache.hadoop.hive.ql.exec.WindowFunctionDescription;
 )
 public class GenericUDAFDenseRank extends GenericUDAFRank {
 
-  static final Log LOG = LogFactory.getLog(GenericUDAFDenseRank.class.getName());
-
   @Override
   protected GenericUDAFAbstractRankEvaluator createEvaluator() {
     return new GenericUDAFDenseRankEvaluator();


[08/50] [abbrv] hive git commit: HIVE-11251: CBO (Calcite Return Path): Extending ExprNodeConverter to consider additional types (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by ga...@apache.org.
HIVE-11251: CBO (Calcite Return Path): Extending ExprNodeConverter to consider additional types (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/hbase-metastore
Commit: 8662d9dae3da1cdbec3ac8c2c4f7d9f12ae5f1f0
Parents: af4aeab
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Wed Jul 15 07:06:14 2015 +0100
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Wed Jul 15 18:31:59 2015 +0100

----------------------------------------------------------------------
 .../calcite/translator/ExprNodeConverter.java      | 17 +++++++++++++++++
 1 file changed, 17 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/8662d9da/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ExprNodeConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ExprNodeConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ExprNodeConverter.java
index bcce74a..4f0db03 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ExprNodeConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ExprNodeConverter.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hive.ql.optimizer.calcite.translator;
 
+import java.math.BigDecimal;
 import java.sql.Date;
 import java.sql.Timestamp;
 import java.util.ArrayList;
@@ -43,6 +44,8 @@ import org.apache.calcite.sql.type.SqlTypeUtil;
 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.HiveIntervalDayTime;
+import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth;
 import org.apache.hadoop.hive.common.type.HiveVarchar;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
 import org.apache.hadoop.hive.ql.optimizer.calcite.translator.ASTConverter.RexVisitor;
@@ -199,6 +202,7 @@ public class ExprNodeConverter extends RexVisitorImpl<ExprNodeDesc> {
       return new ExprNodeConstantDesc(TypeInfoFactory.longTypeInfo, Long.valueOf(((Number) literal
           .getValue3()).longValue()));
     case FLOAT:
+    case REAL:
       return new ExprNodeConstantDesc(TypeInfoFactory.floatTypeInfo,
           Float.valueOf(((Number) literal.getValue3()).floatValue()));
     case DOUBLE:
@@ -207,6 +211,7 @@ public class ExprNodeConverter extends RexVisitorImpl<ExprNodeDesc> {
     case DATE:
       return new ExprNodeConstantDesc(TypeInfoFactory.dateTypeInfo,
         new Date(((Calendar)literal.getValue()).getTimeInMillis()));
+    case TIME:
     case TIMESTAMP: {
       Object value = literal.getValue3();
       if (value instanceof Long) {
@@ -225,6 +230,18 @@ public class ExprNodeConverter extends RexVisitorImpl<ExprNodeDesc> {
     case CHAR:
       return new ExprNodeConstantDesc(TypeInfoFactory.getCharTypeInfo(lType.getPrecision()),
           new HiveChar((String) literal.getValue3(), lType.getPrecision()));
+    case INTERVAL_YEAR_MONTH: {
+      BigDecimal monthsBd = (BigDecimal) literal.getValue();
+      return new ExprNodeConstantDesc(TypeInfoFactory.intervalYearMonthTypeInfo,
+              new HiveIntervalYearMonth(monthsBd.intValue()));
+    }
+    case INTERVAL_DAY_TIME: {
+      BigDecimal millisBd = (BigDecimal) literal.getValue();
+      // Calcite literal is in millis, we need to convert to seconds
+      BigDecimal secsBd = millisBd.divide(BigDecimal.valueOf(1000));
+      return new ExprNodeConstantDesc(TypeInfoFactory.intervalDayTimeTypeInfo,
+              new HiveIntervalDayTime(secsBd));
+    }
     case OTHER:
     default:
       return new ExprNodeConstantDesc(TypeInfoFactory.voidTypeInfo, literal.getValue3());


[10/50] [abbrv] hive git commit: HIVE-11211 : Reset the fields in JoinStatsRule in StatsRulesProcFactory (Pengcheng Xiong, reviewed by Laljo John Pullokkaran)

Posted by ga...@apache.org.
HIVE-11211 : Reset the fields in JoinStatsRule in StatsRulesProcFactory (Pengcheng Xiong, reviewed by Laljo John Pullokkaran)


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

Branch: refs/heads/hbase-metastore
Commit: 42326958148c2558be9c3d4dfe44c9e735704617
Parents: 4d984bd
Author: Hari Subramaniyan <ha...@apache.org>
Authored: Wed Jul 15 13:15:34 2015 -0700
Committer: Hari Subramaniyan <ha...@apache.org>
Committed: Wed Jul 15 13:15:34 2015 -0700

----------------------------------------------------------------------
 .../stats/annotation/StatsRulesProcFactory.java | 42 ++++++++++----------
 1 file changed, 22 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/42326958/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 0982059..376d42c 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
@@ -1013,17 +1013,14 @@ public class StatsRulesProcFactory {
    */
   public static class JoinStatsRule extends DefaultStatsRule implements NodeProcessor {
 
-    private boolean pkfkInferred = false;
-    private long newNumRows = 0;
-    private List<Operator<? extends OperatorDesc>> parents;
-    private CommonJoinOperator<? extends JoinDesc> jop;
-    private int numAttr = 1;
 
     @Override
     public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx,
         Object... nodeOutputs) throws SemanticException {
-      jop = (CommonJoinOperator<? extends JoinDesc>) nd;
-      parents = jop.getParentOperators();
+      long newNumRows = 0;
+      CommonJoinOperator<? extends JoinDesc> jop = (CommonJoinOperator<? extends JoinDesc>) nd;
+      List<Operator<? extends OperatorDesc>> parents = jop.getParentOperators();
+      int numAttr = 1;
       AnnotateStatsProcCtx aspCtx = (AnnotateStatsProcCtx) procCtx;
       HiveConf conf = aspCtx.getConf();
       boolean allStatsAvail = true;
@@ -1062,7 +1059,7 @@ public class StatsRulesProcFactory {
           numAttr = keyExprs.size();
 
           // infer PK-FK relationship in single attribute join case
-          inferPKFKRelationship();
+          long inferredRowCount = inferPKFKRelationship(numAttr, parents, jop);
           // get the join keys from parent ReduceSink operators
           for (int pos = 0; pos < parents.size(); pos++) {
             ReduceSinkOperator parent = (ReduceSinkOperator) jop.getParentOperators().get(pos);
@@ -1149,7 +1146,7 @@ public class StatsRulesProcFactory {
 
           // update join statistics
           stats.setColumnStats(outColStats);
-          long newRowCount = pkfkInferred ? newNumRows : computeNewRowCount(rowCounts, denom);
+          long newRowCount = inferredRowCount !=-1 ? inferredRowCount : computeNewRowCount(rowCounts, denom);
           updateStatsForJoinType(stats, newRowCount, jop, rowCountParents);
           jop.setStatistics(stats);
 
@@ -1180,7 +1177,7 @@ public class StatsRulesProcFactory {
           }
 
           long maxDataSize = parentSizes.get(maxRowIdx);
-          long newNumRows = StatsUtils.safeMult(StatsUtils.safeMult(maxRowCount, (numParents - 1)), joinFactor);
+          newNumRows = StatsUtils.safeMult(StatsUtils.safeMult(maxRowCount, (numParents - 1)), joinFactor);
           long newDataSize = StatsUtils.safeMult(StatsUtils.safeMult(maxDataSize, (numParents - 1)), joinFactor);
           Statistics wcStats = new Statistics();
           wcStats.setNumRows(newNumRows);
@@ -1195,15 +1192,17 @@ public class StatsRulesProcFactory {
       return null;
     }
 
-    private void inferPKFKRelationship() {
+    private long inferPKFKRelationship(int numAttr, List<Operator<? extends OperatorDesc>> parents,
+        CommonJoinOperator<? extends JoinDesc> jop) {
+      long newNumRows = -1;
       if (numAttr == 1) {
         // If numAttr is 1, this means we join on one single key column.
         Map<Integer, ColStatistics> parentsWithPK = getPrimaryKeyCandidates(parents);
 
         // We only allow one single PK.
         if (parentsWithPK.size() != 1) {
-          LOG.debug("STATS-" + jop.toString() + ": detects multiple PK parents.");
-          return;
+          LOG.debug("STATS-" + jop.toString() + ": detects none/multiple PK parents.");
+          return newNumRows;
         }
         Integer pkPos = parentsWithPK.keySet().iterator().next();
         ColStatistics csPK = parentsWithPK.values().iterator().next();
@@ -1215,7 +1214,7 @@ public class StatsRulesProcFactory {
         // csfKs.size() + 1 == parents.size() means we have a single PK and all
         // the rest ops are FKs.
         if (csFKs.size() + 1 == parents.size()) {
-          getSelectivity(parents, pkPos, csPK, csFKs);
+          newNumRows = getCardinality(parents, pkPos, csPK, csFKs, jop);
 
           // some debug information
           if (isDebugEnabled) {
@@ -1236,16 +1235,17 @@ public class StatsRulesProcFactory {
           }
         }
       }
+      return newNumRows;
     }
 
     /**
-     * Get selectivity of reduce sink operators.
+     * Get cardinality of reduce sink operators.
      * @param csPK - ColStatistics for a single primary key
      * @param csFKs - ColStatistics for multiple foreign keys
      */
-    private void getSelectivity(List<Operator<? extends OperatorDesc>> ops, Integer pkPos, ColStatistics csPK,
-        Map<Integer, ColStatistics> csFKs) {
-      this.pkfkInferred = true;
+    private long getCardinality(List<Operator<? extends OperatorDesc>> ops, Integer pkPos,
+        ColStatistics csPK, Map<Integer, ColStatistics> csFKs,
+        CommonJoinOperator<? extends JoinDesc> jop) {
       double pkfkSelectivity = Double.MAX_VALUE;
       int fkInd = -1;
       // 1. We iterate through all the operators that have candidate FKs and
@@ -1290,13 +1290,15 @@ public class StatsRulesProcFactory {
           distinctVals.add(csFK.getCountDistint());
         }
       }
+      long newNumRows;
       if (csFKs.size() == 1) {
         // there is only one FK
-        this.newNumRows = newrows;
+        newNumRows = newrows;
       } else {
         // there is more than one FK
-        this.newNumRows = this.computeNewRowCount(rowCounts, getDenominator(distinctVals));
+        newNumRows = this.computeNewRowCount(rowCounts, getDenominator(distinctVals));
       }
+      return newNumRows;
     }
 
     private float getSelectivitySimpleTree(Operator<? extends OperatorDesc> op) {


[06/50] [abbrv] hive git commit: HIVE-11215: Delete spills only if they exist (Gopal V, reviewed by Matt Mccline)

Posted by ga...@apache.org.
HIVE-11215: Delete spills only if they exist (Gopal V, 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/6e0d4809
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/6e0d4809
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/6e0d4809

Branch: refs/heads/hbase-metastore
Commit: 6e0d4809baed42e25a46373f0a01a3ef421337ad
Parents: 46c76d6
Author: Gopal V <go...@apache.org>
Authored: Tue Jul 14 22:19:02 2015 -0700
Committer: Gopal V <go...@apache.org>
Committed: Tue Jul 14 22:19:02 2015 -0700

----------------------------------------------------------------------
 .../exec/vector/mapjoin/VectorMapJoinRowBytesContainer.java | 9 ++++++---
 1 file changed, 6 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/6e0d4809/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinRowBytesContainer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinRowBytesContainer.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinRowBytesContainer.java
index 32b60d0..d2e980c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinRowBytesContainer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinRowBytesContainer.java
@@ -305,9 +305,12 @@ public class VectorMapJoinRowBytesContainer {
       }
       fileOutputStream = null;
     }
-    try {
-      FileUtil.fullyDelete(parentFile);
-    } catch (Throwable ignored) {
+
+    if (parentFile != null) {
+      try {
+        FileUtil.fullyDelete(parentFile);
+      } catch (Throwable ignored) {
+      }
     }
     parentFile = null;
     tmpFile = null;


[15/50] [abbrv] hive git commit: HIVE-11145 Remove OFFLINE and NO_DROP from tables and partitions (gates, reviewed by Ashutosh Chauhan)

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/list_bucket_query_oneskew_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/list_bucket_query_oneskew_1.q.out b/ql/src/test/results/clientpositive/list_bucket_query_oneskew_1.q.out
index 4a15fae..bec4f6a 100644
--- a/ql/src/test/results/clientpositive/list_bucket_query_oneskew_1.q.out
+++ b/ql/src/test/results/clientpositive/list_bucket_query_oneskew_1.q.out
@@ -137,8 +137,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	fact_daily          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 #### A masked pattern was here ####

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/list_bucket_query_oneskew_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/list_bucket_query_oneskew_2.q.out b/ql/src/test/results/clientpositive/list_bucket_query_oneskew_2.q.out
index 177da44..646f8b8 100644
--- a/ql/src/test/results/clientpositive/list_bucket_query_oneskew_2.q.out
+++ b/ql/src/test/results/clientpositive/list_bucket_query_oneskew_2.q.out
@@ -142,8 +142,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	fact_daily          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 #### A masked pattern was here ####

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/list_bucket_query_oneskew_3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/list_bucket_query_oneskew_3.q.out b/ql/src/test/results/clientpositive/list_bucket_query_oneskew_3.q.out
index d0eb853..fabd0d6 100644
--- a/ql/src/test/results/clientpositive/list_bucket_query_oneskew_3.q.out
+++ b/ql/src/test/results/clientpositive/list_bucket_query_oneskew_3.q.out
@@ -160,8 +160,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	fact_daily          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 #### A masked pattern was here ####

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/merge3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/merge3.q.out b/ql/src/test/results/clientpositive/merge3.q.out
index 2ca40f4..b237631 100644
--- a/ql/src/test/results/clientpositive/merge3.q.out
+++ b/ql/src/test/results/clientpositive/merge3.q.out
@@ -2361,7 +2361,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/orc_analyze.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/orc_analyze.q.out b/ql/src/test/results/clientpositive/orc_analyze.q.out
index a61a2e6..6eb9a93 100644
--- a/ql/src/test/results/clientpositive/orc_analyze.q.out
+++ b/ql/src/test/results/clientpositive/orc_analyze.q.out
@@ -98,7 +98,6 @@ state               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -147,7 +146,6 @@ state               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -196,7 +194,6 @@ state               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -286,7 +283,6 @@ state               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -404,8 +400,6 @@ Partition Value:    	[Ca]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -449,8 +443,6 @@ Partition Value:    	[Or]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -506,8 +498,6 @@ Partition Value:    	[Ca]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -551,8 +541,6 @@ Partition Value:    	[Or]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -608,8 +596,6 @@ Partition Value:    	[Ca]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -653,8 +639,6 @@ Partition Value:    	[Or]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -755,8 +739,6 @@ Partition Value:    	[Ca]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -800,8 +782,6 @@ Partition Value:    	[Or]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -922,8 +902,6 @@ Partition Value:    	[Ca]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -967,8 +945,6 @@ Partition Value:    	[Or]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1024,8 +1000,6 @@ Partition Value:    	[Ca]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1069,8 +1043,6 @@ Partition Value:    	[Or]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1126,8 +1098,6 @@ Partition Value:    	[Ca]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1171,8 +1141,6 @@ Partition Value:    	[Or]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1279,8 +1247,6 @@ Partition Value:    	[Ca]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1324,8 +1290,6 @@ Partition Value:    	[Or]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1491,8 +1455,6 @@ Partition Value:    	[Ca]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1536,8 +1498,6 @@ Partition Value:    	[OH]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   
@@ -1595,8 +1555,6 @@ Partition Value:    	[Ca]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1640,8 +1598,6 @@ Partition Value:    	[OH]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   
@@ -1699,8 +1655,6 @@ Partition Value:    	[Ca]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1744,8 +1698,6 @@ Partition Value:    	[OH]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/orc_create.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/orc_create.q.out b/ql/src/test/results/clientpositive/orc_create.q.out
index bffb58d..e294dec 100644
--- a/ql/src/test/results/clientpositive/orc_create.q.out
+++ b/ql/src/test/results/clientpositive/orc_create.q.out
@@ -62,7 +62,6 @@ strct               	struct<A:string,B:string>
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -113,7 +112,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -167,7 +165,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -211,7 +208,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -265,7 +261,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -316,7 +311,6 @@ strct               	struct<A:string,B:string>
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/parallel_orderby.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/parallel_orderby.q.out b/ql/src/test/results/clientpositive/parallel_orderby.q.out
index 2f4ac8f..4b7de52 100644
--- a/ql/src/test/results/clientpositive/parallel_orderby.q.out
+++ b/ql/src/test/results/clientpositive/parallel_orderby.q.out
@@ -105,7 +105,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -219,7 +218,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/parquet_array_null_element.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/parquet_array_null_element.q.out b/ql/src/test/results/clientpositive/parquet_array_null_element.q.out
index 4243131..387f01e 100644
--- a/ql/src/test/results/clientpositive/parquet_array_null_element.q.out
+++ b/ql/src/test/results/clientpositive/parquet_array_null_element.q.out
@@ -66,7 +66,6 @@ mp                  	map<string,string>
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/parquet_create.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/parquet_create.q.out b/ql/src/test/results/clientpositive/parquet_create.q.out
index 32c4db8..c6d33ff 100644
--- a/ql/src/test/results/clientpositive/parquet_create.q.out
+++ b/ql/src/test/results/clientpositive/parquet_create.q.out
@@ -69,7 +69,6 @@ strct               	struct<A:string,B:string>
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/parquet_partitioned.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/parquet_partitioned.q.out b/ql/src/test/results/clientpositive/parquet_partitioned.q.out
index 2aafb29..3529d70 100644
--- a/ql/src/test/results/clientpositive/parquet_partitioned.q.out
+++ b/ql/src/test/results/clientpositive/parquet_partitioned.q.out
@@ -63,7 +63,6 @@ part                	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/parquet_serde.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/parquet_serde.q.out b/ql/src/test/results/clientpositive/parquet_serde.q.out
index e753180..fb2344a 100644
--- a/ql/src/test/results/clientpositive/parquet_serde.q.out
+++ b/ql/src/test/results/clientpositive/parquet_serde.q.out
@@ -70,8 +70,6 @@ Partition Value:    	[20140330]
 Database:           	default             	 
 Table:              	parquet_mixed_fileformat	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -137,7 +135,6 @@ dateint             	int
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -177,8 +174,6 @@ Partition Value:    	[20140330]
 Database:           	default             	 
 Table:              	parquet_mixed_fileformat	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/part_inherit_tbl_props.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/part_inherit_tbl_props.q.out b/ql/src/test/results/clientpositive/part_inherit_tbl_props.q.out
index 144d89e..82f4750 100644
--- a/ql/src/test/results/clientpositive/part_inherit_tbl_props.q.out
+++ b/ql/src/test/results/clientpositive/part_inherit_tbl_props.q.out
@@ -37,8 +37,6 @@ Partition Value:    	[v1]
 Database:           	default             	 
 Table:              	mytbl               	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	a                   	myval               
 	b                   	yourval             

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/part_inherit_tbl_props_empty.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/part_inherit_tbl_props_empty.q.out b/ql/src/test/results/clientpositive/part_inherit_tbl_props_empty.q.out
index 758712f..e4b8003 100644
--- a/ql/src/test/results/clientpositive/part_inherit_tbl_props_empty.q.out
+++ b/ql/src/test/results/clientpositive/part_inherit_tbl_props_empty.q.out
@@ -33,8 +33,6 @@ Partition Value:    	[v1]
 Database:           	default             	 
 Table:              	mytbl               	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 #### A masked pattern was here ####
 	 	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/part_inherit_tbl_props_with_star.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/part_inherit_tbl_props_with_star.q.out b/ql/src/test/results/clientpositive/part_inherit_tbl_props_with_star.q.out
index 85ad687..106448a 100644
--- a/ql/src/test/results/clientpositive/part_inherit_tbl_props_with_star.q.out
+++ b/ql/src/test/results/clientpositive/part_inherit_tbl_props_with_star.q.out
@@ -37,8 +37,6 @@ Partition Value:    	[v1]
 Database:           	default             	 
 Table:              	mytbl               	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	a                   	myval               
 	b                   	yourval             

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/partition_coltype_literals.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/partition_coltype_literals.q.out b/ql/src/test/results/clientpositive/partition_coltype_literals.q.out
index 9e594c9..86099cb 100644
--- a/ql/src/test/results/clientpositive/partition_coltype_literals.q.out
+++ b/ql/src/test/results/clientpositive/partition_coltype_literals.q.out
@@ -44,8 +44,6 @@ Partition Value:    	[100, 20000, 300000000000]
 Database:           	default             	 
 Table:              	partcoltypenum      	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 #### A masked pattern was here ####
 	 	 
@@ -93,8 +91,6 @@ Partition Value:    	[100, 20000, 300000000000]
 Database:           	default             	 
 Table:              	partcoltypenum      	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 #### A masked pattern was here ####
@@ -148,8 +144,6 @@ Partition Value:    	[100, 20000, 300000000000]
 Database:           	default             	 
 Table:              	partcoltypenum      	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 #### A masked pattern was here ####
@@ -204,8 +198,6 @@ Partition Value:    	[110, 22000, 330000000000]
 Database:           	default             	 
 Table:              	partcoltypenum      	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 #### A masked pattern was here ####
@@ -375,8 +367,6 @@ Partition Value:    	[110, 22000, 330000000000]
 Database:           	default             	 
 Table:              	partcoltypenum      	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 #### A masked pattern was here ####
@@ -428,8 +418,6 @@ Partition Value:    	[110, 22000, 330000000000]
 Database:           	default             	 
 Table:              	partcoltypenum      	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 #### A masked pattern was here ####
@@ -509,8 +497,6 @@ Partition Value:    	[100, 20000, 300000000000]
 Database:           	default             	 
 Table:              	partcoltypenum      	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -562,8 +548,6 @@ Partition Value:    	[100, 20000, 300000000000]
 Database:           	default             	 
 Table:              	partcoltypenum      	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 #### A masked pattern was here ####

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/protectmode2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/protectmode2.q.out b/ql/src/test/results/clientpositive/protectmode2.q.out
index 4ac3e61..ee04393 100644
--- a/ql/src/test/results/clientpositive/protectmode2.q.out
+++ b/ql/src/test/results/clientpositive/protectmode2.q.out
@@ -136,7 +136,6 @@ p                   	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	NO_DROP_CASCADE     	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -180,7 +179,6 @@ p                   	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/rcfile_default_format.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/rcfile_default_format.q.out b/ql/src/test/results/clientpositive/rcfile_default_format.q.out
index e584c8b..ee58509 100644
--- a/ql/src/test/results/clientpositive/rcfile_default_format.q.out
+++ b/ql/src/test/results/clientpositive/rcfile_default_format.q.out
@@ -19,7 +19,6 @@ key                 	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -60,7 +59,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -112,7 +110,6 @@ key                 	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -158,7 +155,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -204,7 +200,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -247,7 +242,6 @@ key                 	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -288,7 +282,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -331,7 +324,6 @@ key                 	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/selectDistinctStar.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/selectDistinctStar.q.out b/ql/src/test/results/clientpositive/selectDistinctStar.q.out
index 8867b8d..45d9882 100644
--- a/ql/src/test/results/clientpositive/selectDistinctStar.q.out
+++ b/ql/src/test/results/clientpositive/selectDistinctStar.q.out
@@ -1363,7 +1363,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 Table Type:         	VIRTUAL_VIEW        	 
 Table Parameters:	 	 
@@ -3794,7 +3793,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 Table Type:         	VIRTUAL_VIEW        	 
 Table Parameters:	 	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/alter_merge_stats_orc.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/alter_merge_stats_orc.q.out b/ql/src/test/results/clientpositive/spark/alter_merge_stats_orc.q.out
index 43cc4ef..cefe069 100644
--- a/ql/src/test/results/clientpositive/spark/alter_merge_stats_orc.q.out
+++ b/ql/src/test/results/clientpositive/spark/alter_merge_stats_orc.q.out
@@ -85,7 +85,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -137,7 +136,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -242,8 +240,6 @@ Partition Value:    	[2011]
 Database:           	default             	 
 Table:              	src_orc_merge_test_part_stat	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	3                   
@@ -293,8 +289,6 @@ Partition Value:    	[2011]
 Database:           	default             	 
 Table:              	src_orc_merge_test_part_stat	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	3                   
@@ -352,8 +346,6 @@ Partition Value:    	[2011]
 Database:           	default             	 
 Table:              	src_orc_merge_test_part_stat	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/bucket5.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/bucket5.q.out b/ql/src/test/results/clientpositive/spark/bucket5.q.out
index 45f2ace..a72db3b 100644
--- a/ql/src/test/results/clientpositive/spark/bucket5.q.out
+++ b/ql/src/test/results/clientpositive/spark/bucket5.q.out
@@ -370,7 +370,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/ctas.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/ctas.q.out b/ql/src/test/results/clientpositive/spark/ctas.q.out
index cebe3ab..6bb2f76 100644
--- a/ql/src/test/results/clientpositive/spark/ctas.q.out
+++ b/ql/src/test/results/clientpositive/spark/ctas.q.out
@@ -142,7 +142,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -286,7 +285,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -430,7 +428,6 @@ conb                	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -495,7 +492,6 @@ conb                	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -640,7 +636,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/infer_bucket_sort_bucketed_table.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/infer_bucket_sort_bucketed_table.q.out b/ql/src/test/results/clientpositive/spark/infer_bucket_sort_bucketed_table.q.out
index 2852ae9..5573c0a 100644
--- a/ql/src/test/results/clientpositive/spark/infer_bucket_sort_bucketed_table.q.out
+++ b/ql/src/test/results/clientpositive/spark/infer_bucket_sort_bucketed_table.q.out
@@ -49,8 +49,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table_bucketed 	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	3                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/infer_bucket_sort_convert_join.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/infer_bucket_sort_convert_join.q.out b/ql/src/test/results/clientpositive/spark/infer_bucket_sort_convert_join.q.out
index 3d4eb18..05bbc26 100644
--- a/ql/src/test/results/clientpositive/spark/infer_bucket_sort_convert_join.q.out
+++ b/ql/src/test/results/clientpositive/spark/infer_bucket_sort_convert_join.q.out
@@ -49,8 +49,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -110,8 +108,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	0                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/infer_bucket_sort_map_operators.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/infer_bucket_sort_map_operators.q.out b/ql/src/test/results/clientpositive/spark/infer_bucket_sort_map_operators.q.out
index 94adb3d..f39cd57 100644
--- a/ql/src/test/results/clientpositive/spark/infer_bucket_sort_map_operators.q.out
+++ b/ql/src/test/results/clientpositive/spark/infer_bucket_sort_map_operators.q.out
@@ -144,8 +144,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table_out      	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   
@@ -315,8 +313,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table_out      	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	4                   
@@ -432,8 +428,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table_out      	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   
@@ -571,8 +565,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table_out      	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	4                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/infer_bucket_sort_merge.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/infer_bucket_sort_merge.q.out b/ql/src/test/results/clientpositive/spark/infer_bucket_sort_merge.q.out
index d947eb5..4db9e35 100644
--- a/ql/src/test/results/clientpositive/spark/infer_bucket_sort_merge.q.out
+++ b/ql/src/test/results/clientpositive/spark/infer_bucket_sort_merge.q.out
@@ -49,8 +49,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   
@@ -104,8 +102,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/infer_bucket_sort_num_buckets.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/infer_bucket_sort_num_buckets.q.out b/ql/src/test/results/clientpositive/spark/infer_bucket_sort_num_buckets.q.out
index c57530e..b9dc290 100644
--- a/ql/src/test/results/clientpositive/spark/infer_bucket_sort_num_buckets.q.out
+++ b/ql/src/test/results/clientpositive/spark/infer_bucket_sort_num_buckets.q.out
@@ -154,8 +154,6 @@ Partition Value:    	[2008-04-08, 0]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   
@@ -196,8 +194,6 @@ Partition Value:    	[2008-04-08, 1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/infer_bucket_sort_reducers_power_two.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/infer_bucket_sort_reducers_power_two.q.out b/ql/src/test/results/clientpositive/spark/infer_bucket_sort_reducers_power_two.q.out
index c9072e5..15b7992 100644
--- a/ql/src/test/results/clientpositive/spark/infer_bucket_sort_reducers_power_two.q.out
+++ b/ql/src/test/results/clientpositive/spark/infer_bucket_sort_reducers_power_two.q.out
@@ -49,8 +49,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	5                   
@@ -104,8 +102,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	5                   
@@ -159,8 +155,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	4                   
@@ -214,8 +208,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	7                   
@@ -269,8 +261,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	5                   
@@ -326,8 +316,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_table          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	4                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/list_bucket_dml_10.q.java1.7.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/list_bucket_dml_10.q.java1.7.out b/ql/src/test/results/clientpositive/spark/list_bucket_dml_10.q.java1.7.out
index d765eaf..2f84758 100644
--- a/ql/src/test/results/clientpositive/spark/list_bucket_dml_10.q.java1.7.out
+++ b/ql/src/test/results/clientpositive/spark/list_bucket_dml_10.q.java1.7.out
@@ -255,8 +255,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	list_bucketing_static_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	4                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/list_bucket_dml_10.q.java1.8.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/list_bucket_dml_10.q.java1.8.out b/ql/src/test/results/clientpositive/spark/list_bucket_dml_10.q.java1.8.out
index 52a76e6..12f41eb 100644
--- a/ql/src/test/results/clientpositive/spark/list_bucket_dml_10.q.java1.8.out
+++ b/ql/src/test/results/clientpositive/spark/list_bucket_dml_10.q.java1.8.out
@@ -255,8 +255,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	list_bucketing_static_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	4                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/list_bucket_dml_2.q.java1.7.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/list_bucket_dml_2.q.java1.7.out b/ql/src/test/results/clientpositive/spark/list_bucket_dml_2.q.java1.7.out
index e38ccf8..11ffff8 100644
--- a/ql/src/test/results/clientpositive/spark/list_bucket_dml_2.q.java1.7.out
+++ b/ql/src/test/results/clientpositive/spark/list_bucket_dml_2.q.java1.7.out
@@ -315,8 +315,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	list_bucketing_static_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	6                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/list_bucket_dml_2.q.java1.8.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/list_bucket_dml_2.q.java1.8.out b/ql/src/test/results/clientpositive/spark/list_bucket_dml_2.q.java1.8.out
index dade6bb..23dc6a3 100644
--- a/ql/src/test/results/clientpositive/spark/list_bucket_dml_2.q.java1.8.out
+++ b/ql/src/test/results/clientpositive/spark/list_bucket_dml_2.q.java1.8.out
@@ -315,8 +315,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	list_bucketing_static_part	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	6                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/list_bucket_dml_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/list_bucket_dml_2.q.out b/ql/src/test/results/clientpositive/spark/list_bucket_dml_2.q.out
index 17bd5bb..3ee9b5a 100644
Binary files a/ql/src/test/results/clientpositive/spark/list_bucket_dml_2.q.out and b/ql/src/test/results/clientpositive/spark/list_bucket_dml_2.q.out differ

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/orc_analyze.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/orc_analyze.q.out b/ql/src/test/results/clientpositive/spark/orc_analyze.q.out
index 60a23d4..121142e 100644
--- a/ql/src/test/results/clientpositive/spark/orc_analyze.q.out
+++ b/ql/src/test/results/clientpositive/spark/orc_analyze.q.out
@@ -98,7 +98,6 @@ state               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -188,7 +187,6 @@ state               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -306,8 +304,6 @@ Partition Value:    	[Ca]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -351,8 +347,6 @@ Partition Value:    	[Or]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -453,8 +447,6 @@ Partition Value:    	[Ca]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -498,8 +490,6 @@ Partition Value:    	[Or]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -620,8 +610,6 @@ Partition Value:    	[Ca]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -665,8 +653,6 @@ Partition Value:    	[Or]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -773,8 +759,6 @@ Partition Value:    	[Ca]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -818,8 +802,6 @@ Partition Value:    	[Or]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -985,8 +967,6 @@ Partition Value:    	[Ca]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1030,8 +1010,6 @@ Partition Value:    	[OH]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/parallel_orderby.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/parallel_orderby.q.out b/ql/src/test/results/clientpositive/spark/parallel_orderby.q.out
index 03314ea..308b82c 100644
--- a/ql/src/test/results/clientpositive/spark/parallel_orderby.q.out
+++ b/ql/src/test/results/clientpositive/spark/parallel_orderby.q.out
@@ -111,7 +111,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -225,7 +224,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/stats1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/stats1.q.out b/ql/src/test/results/clientpositive/spark/stats1.q.out
index ec2edc4..ab1f993 100644
--- a/ql/src/test/results/clientpositive/spark/stats1.q.out
+++ b/ql/src/test/results/clientpositive/spark/stats1.q.out
@@ -170,7 +170,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -220,7 +219,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/stats10.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/stats10.q.out b/ql/src/test/results/clientpositive/spark/stats10.q.out
index c840ab7..9c5090f 100644
--- a/ql/src/test/results/clientpositive/spark/stats10.q.out
+++ b/ql/src/test/results/clientpositive/spark/stats10.q.out
@@ -422,8 +422,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	bucket3_1           	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   
@@ -463,8 +461,6 @@ Partition Value:    	[2]
 Database:           	default             	 
 Table:              	bucket3_1           	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   
@@ -502,7 +498,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/stats12.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/stats12.q.out b/ql/src/test/results/clientpositive/spark/stats12.q.out
index db575df..4a5f075 100644
--- a/ql/src/test/results/clientpositive/spark/stats12.q.out
+++ b/ql/src/test/results/clientpositive/spark/stats12.q.out
@@ -208,7 +208,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -247,8 +246,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -289,8 +286,6 @@ Partition Value:    	[2008-04-08, 12]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -331,8 +326,6 @@ Partition Value:    	[2008-04-09, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   
@@ -373,8 +366,6 @@ Partition Value:    	[2008-04-09, 12]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/stats13.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/stats13.q.out b/ql/src/test/results/clientpositive/spark/stats13.q.out
index f38f876..452d4bc 100644
--- a/ql/src/test/results/clientpositive/spark/stats13.q.out
+++ b/ql/src/test/results/clientpositive/spark/stats13.q.out
@@ -158,7 +158,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -197,8 +196,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -239,8 +236,6 @@ Partition Value:    	[2008-04-08, 12]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   
@@ -281,8 +276,6 @@ Partition Value:    	[2008-04-09, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   
@@ -323,8 +316,6 @@ Partition Value:    	[2008-04-09, 12]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   
@@ -371,7 +362,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/stats14.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/stats14.q.out b/ql/src/test/results/clientpositive/spark/stats14.q.out
index f12b136..f34720d 100644
--- a/ql/src/test/results/clientpositive/spark/stats14.q.out
+++ b/ql/src/test/results/clientpositive/spark/stats14.q.out
@@ -38,7 +38,6 @@ value               	string              	default
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -142,7 +141,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -181,8 +179,6 @@ Partition Value:    	[2010-04-08, 11]
 Database:           	default             	 
 Table:              	stats_part          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -223,8 +219,6 @@ Partition Value:    	[2010-04-08, 12]
 Database:           	default             	 
 Table:              	stats_part          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -283,7 +277,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/stats15.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/stats15.q.out b/ql/src/test/results/clientpositive/spark/stats15.q.out
index a60dee2..aad2e3a 100644
--- a/ql/src/test/results/clientpositive/spark/stats15.q.out
+++ b/ql/src/test/results/clientpositive/spark/stats15.q.out
@@ -38,7 +38,6 @@ value               	string              	default
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -142,7 +141,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -181,8 +179,6 @@ Partition Value:    	[2010-04-08, 11]
 Database:           	default             	 
 Table:              	stats_part          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -223,8 +219,6 @@ Partition Value:    	[2010-04-08, 12]
 Database:           	default             	 
 Table:              	stats_part          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -283,7 +277,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/stats16.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/stats16.q.out b/ql/src/test/results/clientpositive/spark/stats16.q.out
index 3f0f2ea..2e3cadb 100644
--- a/ql/src/test/results/clientpositive/spark/stats16.q.out
+++ b/ql/src/test/results/clientpositive/spark/stats16.q.out
@@ -24,7 +24,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -73,7 +72,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/stats18.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/stats18.q.out b/ql/src/test/results/clientpositive/spark/stats18.q.out
index a061846..a7d6ab8 100644
--- a/ql/src/test/results/clientpositive/spark/stats18.q.out
+++ b/ql/src/test/results/clientpositive/spark/stats18.q.out
@@ -44,8 +44,6 @@ Partition Value:    	[2010-04-08, 13]
 Database:           	default             	 
 Table:              	stats_part          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -94,8 +92,6 @@ Partition Value:    	[2010-04-08, 13]
 Database:           	default             	 
 Table:              	stats_part          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/stats2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/stats2.q.out b/ql/src/test/results/clientpositive/spark/stats2.q.out
index ddc8226..404e34b 100644
--- a/ql/src/test/results/clientpositive/spark/stats2.q.out
+++ b/ql/src/test/results/clientpositive/spark/stats2.q.out
@@ -99,7 +99,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -182,7 +181,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/stats20.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/stats20.q.out b/ql/src/test/results/clientpositive/spark/stats20.q.out
index 4ac7bc5..d7e52b4 100644
--- a/ql/src/test/results/clientpositive/spark/stats20.q.out
+++ b/ql/src/test/results/clientpositive/spark/stats20.q.out
@@ -39,7 +39,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -89,7 +88,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/stats3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/stats3.q.out b/ql/src/test/results/clientpositive/spark/stats3.q.out
index dd3a95b..2afb76e 100644
--- a/ql/src/test/results/clientpositive/spark/stats3.q.out
+++ b/ql/src/test/results/clientpositive/spark/stats3.q.out
@@ -82,7 +82,6 @@ col1                	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -227,7 +226,6 @@ pcol2               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/stats5.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/stats5.q.out b/ql/src/test/results/clientpositive/spark/stats5.q.out
index b61101d..9748469 100644
--- a/ql/src/test/results/clientpositive/spark/stats5.q.out
+++ b/ql/src/test/results/clientpositive/spark/stats5.q.out
@@ -52,7 +52,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/stats6.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/stats6.q.out b/ql/src/test/results/clientpositive/spark/stats6.q.out
index b4435f2..a387075 100644
--- a/ql/src/test/results/clientpositive/spark/stats6.q.out
+++ b/ql/src/test/results/clientpositive/spark/stats6.q.out
@@ -79,8 +79,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -121,8 +119,6 @@ Partition Value:    	[2008-04-08, 12]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -163,8 +159,6 @@ Partition Value:    	[2008-04-09, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   
@@ -205,8 +199,6 @@ Partition Value:    	[2008-04-09, 12]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   
@@ -245,7 +237,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/stats7.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/stats7.q.out b/ql/src/test/results/clientpositive/spark/stats7.q.out
index 0e49e33..0e095fc 100644
--- a/ql/src/test/results/clientpositive/spark/stats7.q.out
+++ b/ql/src/test/results/clientpositive/spark/stats7.q.out
@@ -93,8 +93,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -135,8 +133,6 @@ Partition Value:    	[2008-04-08, 12]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -175,7 +171,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/stats8.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/stats8.q.out b/ql/src/test/results/clientpositive/spark/stats8.q.out
index 0213e10..3f4ed63 100644
--- a/ql/src/test/results/clientpositive/spark/stats8.q.out
+++ b/ql/src/test/results/clientpositive/spark/stats8.q.out
@@ -89,8 +89,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -129,7 +127,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -202,8 +199,6 @@ Partition Value:    	[2008-04-08, 12]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -278,8 +273,6 @@ Partition Value:    	[2008-04-09, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -354,8 +347,6 @@ Partition Value:    	[2008-04-09, 12]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -442,8 +433,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -484,8 +473,6 @@ Partition Value:    	[2008-04-08, 12]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -526,8 +513,6 @@ Partition Value:    	[2008-04-09, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -568,8 +553,6 @@ Partition Value:    	[2008-04-09, 12]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -608,7 +591,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/stats9.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/stats9.q.out b/ql/src/test/results/clientpositive/spark/stats9.q.out
index 2c7daea..7eae829 100644
--- a/ql/src/test/results/clientpositive/spark/stats9.q.out
+++ b/ql/src/test/results/clientpositive/spark/stats9.q.out
@@ -60,7 +60,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/spark/stats_counter.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/stats_counter.q.out b/ql/src/test/results/clientpositive/spark/stats_counter.q.out
index e2980e8..8b3dcea 100644
--- a/ql/src/test/results/clientpositive/spark/stats_counter.q.out
+++ b/ql/src/test/results/clientpositive/spark/stats_counter.q.out
@@ -32,7 +32,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -80,7 +79,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 


[04/50] [abbrv] hive git commit: HIVE-11252 : CBO (Calcite Return Path): DUMMY project in plan (Jesus Camacho Rodriguez via Ashutosh Chauhan)

Posted by ga...@apache.org.
HIVE-11252 : CBO (Calcite Return Path): DUMMY project in plan (Jesus Camacho Rodriguez via Ashutosh Chauhan)


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

Branch: refs/heads/hbase-metastore
Commit: 999e0e3616525d77cf46c5865f9981b5a6b5609a
Parents: 90a2cf9
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Tue Jul 14 08:22:00 2015 +0700
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Tue Jul 14 11:18:33 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hive/ql/parse/CalcitePlanner.java    | 64 +++++++++-----------
 1 file changed, 30 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/999e0e36/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 84bb951..1ea236b 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
@@ -863,38 +863,20 @@ public class CalcitePlanner extends SemanticAnalyzer {
       calciteOptimizedPlan = hepPlanner.findBestExp();
 
       // 4. Run rule to try to remove projects on top of join operators
-      hepPgmBldr = new HepProgramBuilder().addMatchOrder(HepMatchOrder.BOTTOM_UP);
-      hepPgmBldr.addRuleInstance(HiveJoinCommuteRule.INSTANCE);
-      hepPlanner = new HepPlanner(hepPgmBldr.build());
-      hepPlanner.registerMetadataProviders(list);
-      cluster.setMetadataProvider(new CachingRelMetadataProvider(chainedProvider, hepPlanner));
-      hepPlanner.setRoot(calciteOptimizedPlan);
-      calciteOptimizedPlan = hepPlanner.findBestExp();
+      calciteOptimizedPlan = hepPlan(calciteOptimizedPlan, false, mdProvider.getMetadataProvider(),
+              HepMatchOrder.BOTTOM_UP, HiveJoinCommuteRule.INSTANCE);
 
       // 5. Run rule to fix windowing issue when it is done over
       // aggregation columns (HIVE-10627)
-      hepPgmBldr = new HepProgramBuilder().addMatchOrder(HepMatchOrder.BOTTOM_UP);
-      hepPgmBldr.addRuleInstance(HiveWindowingFixRule.INSTANCE);
-      hepPlanner = new HepPlanner(hepPgmBldr.build());
-      hepPlanner.registerMetadataProviders(list);
-      cluster.setMetadataProvider(new CachingRelMetadataProvider(chainedProvider, hepPlanner));
-      hepPlanner.setRoot(calciteOptimizedPlan);
-      calciteOptimizedPlan = hepPlanner.findBestExp();
+      calciteOptimizedPlan = hepPlan(calciteOptimizedPlan, false, mdProvider.getMetadataProvider(),
+              HepMatchOrder.BOTTOM_UP, HiveWindowingFixRule.INSTANCE);
 
       // 6. Run rules to aid in translation from Calcite tree to Hive tree
       if (HiveConf.getBoolVar(conf, ConfVars.HIVE_CBO_RETPATH_HIVEOP)) {
         // 6.1. Merge join into multijoin operators (if possible)
-        hepPgmBldr = new HepProgramBuilder().addMatchOrder(HepMatchOrder.BOTTOM_UP);
-        hepPgmBldr.addRuleInstance(HiveJoinToMultiJoinRule.INSTANCE);
-        hepPgmBldr = hepPgmBldr.addRuleCollection(ImmutableList.of(
-                HiveJoinProjectTransposeRule.BOTH_PROJECT,
-                HiveJoinToMultiJoinRule.INSTANCE,
-                HiveProjectMergeRule.INSTANCE));
-        hepPlanner = new HepPlanner(hepPgmBldr.build());
-        hepPlanner.registerMetadataProviders(list);
-        cluster.setMetadataProvider(new CachingRelMetadataProvider(chainedProvider, hepPlanner));
-        hepPlanner.setRoot(calciteOptimizedPlan);
-        calciteOptimizedPlan = hepPlanner.findBestExp();
+        calciteOptimizedPlan = hepPlan(calciteOptimizedPlan, true, mdProvider.getMetadataProvider(),
+                HepMatchOrder.BOTTOM_UP, HiveJoinProjectTransposeRule.BOTH_PROJECT,
+                HiveJoinToMultiJoinRule.INSTANCE, HiveProjectMergeRule.INSTANCE);
         // The previous rules can pull up projections through join operators,
         // thus we run the field trimmer again to push them back down
         HiveRelFieldTrimmer fieldTrimmer = new HiveRelFieldTrimmer(null, HiveProject.DEFAULT_PROJECT_FACTORY,
@@ -902,16 +884,14 @@ public class CalcitePlanner extends SemanticAnalyzer {
             HiveSemiJoin.HIVE_SEMIJOIN_FACTORY, HiveSort.HIVE_SORT_REL_FACTORY,
             HiveAggregate.HIVE_AGGR_REL_FACTORY, HiveUnion.UNION_REL_FACTORY);
         calciteOptimizedPlan = fieldTrimmer.trim(calciteOptimizedPlan);
+        calciteOptimizedPlan = hepPlan(calciteOptimizedPlan, false, mdProvider.getMetadataProvider(),
+                HepMatchOrder.BOTTOM_UP, ProjectRemoveRule.INSTANCE,
+                new ProjectMergeRule(false, HiveProject.DEFAULT_PROJECT_FACTORY));
 
         // 6.2.  Introduce exchange operators below join/multijoin operators
-        hepPgmBldr = new HepProgramBuilder().addMatchOrder(HepMatchOrder.BOTTOM_UP);
-        hepPgmBldr.addRuleInstance(HiveInsertExchange4JoinRule.EXCHANGE_BELOW_JOIN);
-        hepPgmBldr.addRuleInstance(HiveInsertExchange4JoinRule.EXCHANGE_BELOW_MULTIJOIN);
-        hepPlanner = new HepPlanner(hepPgmBldr.build());
-        hepPlanner.registerMetadataProviders(list);
-        cluster.setMetadataProvider(new CachingRelMetadataProvider(chainedProvider, hepPlanner));
-        hepPlanner.setRoot(calciteOptimizedPlan);
-        calciteOptimizedPlan = hepPlanner.findBestExp();
+        calciteOptimizedPlan = hepPlan(calciteOptimizedPlan, false, mdProvider.getMetadataProvider(),
+                HepMatchOrder.BOTTOM_UP, HiveInsertExchange4JoinRule.EXCHANGE_BELOW_JOIN,
+                HiveInsertExchange4JoinRule.EXCHANGE_BELOW_MULTIJOIN);
       }
 
       if (LOG.isDebugEnabled() && !conf.getBoolVar(ConfVars.HIVE_IN_TEST)) {
@@ -1006,11 +986,27 @@ public class CalcitePlanner extends SemanticAnalyzer {
      */
     private RelNode hepPlan(RelNode basePlan, boolean followPlanChanges,
         RelMetadataProvider mdProvider, RelOptRule... rules) {
+      return hepPlan(basePlan, followPlanChanges, mdProvider,
+              HepMatchOrder.TOP_DOWN, rules);
+    }
+
+    /**
+     * Run the HEP Planner with the given rule set.
+     *
+     * @param basePlan
+     * @param followPlanChanges
+     * @param mdProvider
+     * @param order
+     * @param rules
+     * @return optimized RelNode
+     */
+    private RelNode hepPlan(RelNode basePlan, boolean followPlanChanges, RelMetadataProvider mdProvider,
+            HepMatchOrder order, RelOptRule... rules) {
 
       RelNode optimizedRelNode = basePlan;
       HepProgramBuilder programBuilder = new HepProgramBuilder();
       if (followPlanChanges) {
-        programBuilder.addMatchOrder(HepMatchOrder.TOP_DOWN);
+        programBuilder.addMatchOrder(order);
         programBuilder = programBuilder.addRuleCollection(ImmutableList.copyOf(rules));
       } else {
         // TODO: Should this be also TOP_DOWN?


[18/50] [abbrv] hive git commit: HIVE-11145 Remove OFFLINE and NO_DROP from tables and partitions (gates, reviewed by Ashutosh Chauhan)

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java
index 9e1ac80..2dabce2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java
@@ -18,14 +18,6 @@
 
 package org.apache.hadoop.hive.ql.plan;
 
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Order;
 import org.apache.hadoop.hive.ql.exec.Utilities;
@@ -34,6 +26,14 @@ import org.apache.hadoop.hive.ql.parse.ParseUtils;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.Explain.Level;
 
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
 /**
  * AlterTableDesc.
  *
@@ -51,8 +51,7 @@ public class AlterTableDesc extends DDLDesc implements Serializable {
     ADDPROPS("add props"), DROPPROPS("drop props"), ADDSERDE("add serde"), ADDSERDEPROPS("add serde props"),
     ADDFILEFORMAT("add fileformat"), ADDCLUSTERSORTCOLUMN("add cluster sort column"),
     RENAMECOLUMN("rename column"), ADDPARTITION("add partition"), TOUCH("touch"), ARCHIVE("archieve"),
-    UNARCHIVE("unarchieve"), ALTERPROTECTMODE("alter protect mode"),
-    ALTERPARTITIONPROTECTMODE("alter partition protect mode"), ALTERLOCATION("alter location"),
+    UNARCHIVE("unarchieve"), ALTERLOCATION("alter location"),
     DROPPARTITION("drop partition"), RENAMEPARTITION("rename partition"), ADDSKEWEDBY("add skew column"),
     ALTERSKEWEDLOCATION("alter skew location"), ALTERBUCKETNUM("alter bucket number"),
     ALTERPARTITION("alter partition"), COMPACT("compact"),
@@ -72,7 +71,6 @@ public class AlterTableDesc extends DDLDesc implements Serializable {
       new HashSet<AlterTableDesc.AlterTableTypes>();
 
   static {
-    alterTableTypesWithPartialSpec.add(AlterTableDesc.AlterTableTypes.ALTERPROTECTMODE);
     alterTableTypesWithPartialSpec.add(AlterTableDesc.AlterTableTypes.ADDCOLS);
     alterTableTypesWithPartialSpec.add(AlterTableDesc.AlterTableTypes.REPLACECOLS);
     alterTableTypesWithPartialSpec.add(AlterTableDesc.AlterTableTypes.RENAMECOLUMN);

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/java/org/apache/hadoop/hive/ql/plan/DropTableDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/DropTableDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/DropTableDesc.java
index bb0e7f7..62c8f7e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/DropTableDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/DropTableDesc.java
@@ -18,12 +18,13 @@
 
 package org.apache.hadoop.hive.ql.plan;
 
+import org.apache.hadoop.hive.ql.parse.ReplicationSpec;
+import org.apache.hadoop.hive.ql.plan.Explain.Level;
+
 import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
-import org.apache.hadoop.hive.ql.parse.ReplicationSpec;
-import org.apache.hadoop.hive.ql.plan.Explain.Level;
 
 /**
  * DropTableDesc.
@@ -55,7 +56,6 @@ public class DropTableDesc extends DDLDesc implements Serializable {
   boolean expectView;
   boolean ifExists;
   boolean ifPurge;
-  boolean ignoreProtection;
   ReplicationSpec replicationSpec;
 
   public DropTableDesc() {
@@ -73,13 +73,11 @@ public class DropTableDesc extends DDLDesc implements Serializable {
     this.expectView = expectView;
     this.ifExists = ifExists;
     this.ifPurge = ifPurge;
-    this.ignoreProtection = false;
     this.replicationSpec = replicationSpec;
   }
 
   public DropTableDesc(String tableName, Map<Integer, List<ExprNodeGenericFuncDesc>> partSpecs,
-      boolean expectView, boolean ignoreProtection, boolean ifPurge,
-      ReplicationSpec replicationSpec) {
+      boolean expectView, boolean ifPurge, ReplicationSpec replicationSpec) {
     this.tableName = tableName;
     this.partSpecs = new ArrayList<PartSpec>(partSpecs.size());
     for (Map.Entry<Integer, List<ExprNodeGenericFuncDesc>> partSpec : partSpecs.entrySet()) {
@@ -88,7 +86,6 @@ public class DropTableDesc extends DDLDesc implements Serializable {
         this.partSpecs.add(new PartSpec(expr, prefixLength));
       }
     }
-    this.ignoreProtection = ignoreProtection;
     this.expectView = expectView;
     this.ifPurge = ifPurge;
     this.replicationSpec = replicationSpec;
@@ -118,21 +115,6 @@ public class DropTableDesc extends DDLDesc implements Serializable {
   }
 
   /**
-   * @return whether or not protection will be ignored for the partition
-   */
-  public boolean getIgnoreProtection() {
-    return ignoreProtection;
-  }
-
-  /**
-   * @param ignoreProtection
-   *          set whether or not protection will be ignored for the partition
-   */
-   public void setIgnoreProtection(boolean ignoreProtection) {
-     this.ignoreProtection = ignoreProtection;
-   }
-
-  /**
    * @return whether to expect a view being dropped
    */
   public boolean getExpectView() {

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/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 75cdf16..df37832 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
@@ -93,8 +93,6 @@ public enum HiveOperation {
   SHOW_ROLES("SHOW_ROLES", null, null),
   SHOW_ROLE_PRINCIPALS("SHOW_ROLE_PRINCIPALS", null, null),
   SHOW_ROLE_GRANT("SHOW_ROLE_GRANT", null, null),
-  ALTERTABLE_PROTECTMODE("ALTERTABLE_PROTECTMODE", new Privilege[]{Privilege.ALTER_METADATA}, null),
-  ALTERPARTITION_PROTECTMODE("ALTERPARTITION_PROTECTMODE", new Privilege[]{Privilege.ALTER_METADATA}, null),
   ALTERTABLE_FILEFORMAT("ALTERTABLE_FILEFORMAT", new Privilege[]{Privilege.ALTER_METADATA}, null),
   ALTERPARTITION_FILEFORMAT("ALTERPARTITION_FILEFORMAT", new Privilege[]{Privilege.ALTER_METADATA}, null),
   ALTERTABLE_LOCATION("ALTERTABLE_LOCATION", new Privilege[]{Privilege.ALTER_DATA}, null),

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/queries/clientnegative/alter_partition_invalidspec.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/alter_partition_invalidspec.q b/ql/src/test/queries/clientnegative/alter_partition_invalidspec.q
deleted file mode 100644
index 8cbb25c..0000000
--- a/ql/src/test/queries/clientnegative/alter_partition_invalidspec.q
+++ /dev/null
@@ -1,8 +0,0 @@
--- Create table
-create table if not exists alter_part_invalidspec(key string, value string ) partitioned by (year string, month string) stored as textfile ;
-
--- Load data
-load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_invalidspec partition (year='1996', month='10');
-load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_invalidspec partition (year='1996', month='12');
-
-alter table alter_part_invalidspec partition (year='1997') enable no_drop;

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/queries/clientnegative/alter_partition_nodrop.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/alter_partition_nodrop.q b/ql/src/test/queries/clientnegative/alter_partition_nodrop.q
deleted file mode 100644
index 3c0ff02..0000000
--- a/ql/src/test/queries/clientnegative/alter_partition_nodrop.q
+++ /dev/null
@@ -1,9 +0,0 @@
--- Create table
-create table if not exists alter_part_nodrop_part(key string, value string ) partitioned by (year string, month string) stored as textfile ;
-
--- Load data
-load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_nodrop_part partition (year='1996', month='10');
-load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_nodrop_part partition (year='1996', month='12');
-
-alter table alter_part_nodrop_part partition (year='1996') enable no_drop;
-alter table alter_part_nodrop_part drop partition (year='1996');

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/queries/clientnegative/alter_partition_nodrop_table.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/alter_partition_nodrop_table.q b/ql/src/test/queries/clientnegative/alter_partition_nodrop_table.q
deleted file mode 100644
index f2135b1..0000000
--- a/ql/src/test/queries/clientnegative/alter_partition_nodrop_table.q
+++ /dev/null
@@ -1,9 +0,0 @@
--- Create table
-create table if not exists alter_part_nodrop_table(key string, value string ) partitioned by (year string, month string) stored as textfile ;
-
--- Load data
-load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_nodrop_table partition (year='1996', month='10');
-load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_nodrop_table partition (year='1996', month='12');
-
-alter table alter_part_nodrop_table partition (year='1996') enable no_drop;
-drop table alter_part_nodrop_table;

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/queries/clientnegative/alter_partition_offline.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/alter_partition_offline.q b/ql/src/test/queries/clientnegative/alter_partition_offline.q
deleted file mode 100644
index 7376d8b..0000000
--- a/ql/src/test/queries/clientnegative/alter_partition_offline.q
+++ /dev/null
@@ -1,11 +0,0 @@
--- create table
-create table if not exists alter_part_offline (key string, value string ) partitioned by (year string, month string) stored as textfile ;
-
--- Load data
-load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_offline partition (year='1996', month='10');
-load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_offline partition (year='1996', month='12');
-
-alter table alter_part_offline partition (year='1996') disable offline;
-select * from alter_part_offline where year = '1996';
-alter table alter_part_offline partition (year='1996') enable offline;
-select * from alter_part_offline where year = '1996';

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/queries/clientnegative/drop_table_failure3.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/drop_table_failure3.q b/ql/src/test/queries/clientnegative/drop_table_failure3.q
deleted file mode 100644
index 534ce0b..0000000
--- a/ql/src/test/queries/clientnegative/drop_table_failure3.q
+++ /dev/null
@@ -1,12 +0,0 @@
-create database dtf3;
-use dtf3; 
-
-create table drop_table_failure_temp(col STRING) partitioned by (p STRING);
-
-alter table drop_table_failure_temp add partition (p ='p1');
-alter table drop_table_failure_temp add partition (p ='p2');
-alter table drop_table_failure_temp add partition (p ='p3');
-
-alter table drop_table_failure_temp partition (p ='p3') ENABLE NO_DROP;
-
-drop table drop_table_failure_temp;

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/queries/clientnegative/protectmode_part.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/protectmode_part.q b/ql/src/test/queries/clientnegative/protectmode_part.q
deleted file mode 100644
index 5415999..0000000
--- a/ql/src/test/queries/clientnegative/protectmode_part.q
+++ /dev/null
@@ -1,15 +0,0 @@
--- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode
-
-drop table tbl_protectmode3;
-
-create table tbl_protectmode3  (col string) partitioned by (p string);
-alter table tbl_protectmode3 add partition (p='p1');
-alter table tbl_protectmode3 add partition (p='p2');
-
-select * from tbl_protectmode3 where p='p1';
-select * from tbl_protectmode3 where p='p2';
-
-alter table tbl_protectmode3 partition (p='p1') enable offline;
-
-select * from tbl_protectmode3 where p='p2';
-select * from tbl_protectmode3 where p='p1';

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/queries/clientnegative/protectmode_part1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/protectmode_part1.q b/ql/src/test/queries/clientnegative/protectmode_part1.q
deleted file mode 100644
index 99256da..0000000
--- a/ql/src/test/queries/clientnegative/protectmode_part1.q
+++ /dev/null
@@ -1,21 +0,0 @@
--- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode
-
-drop table tbl_protectmode5;
-
-create table tbl_protectmode5_1 (col string);
-
-create table tbl_protectmode5  (col string) partitioned by (p string);
-alter table tbl_protectmode5 add partition (p='p1');
-alter table tbl_protectmode5 add partition (p='p2');
-
-insert overwrite table tbl_protectmode5_1
-select col from tbl_protectmode5 where p='p1';
-insert overwrite table tbl_protectmode5_1
-select col from tbl_protectmode5 where p='p2';
-
-alter table tbl_protectmode5 partition (p='p1') enable offline;
-
-insert overwrite table tbl_protectmode5_1
-select col from tbl_protectmode5 where p='p2';
-insert overwrite table tbl_protectmode5_1
-select col from tbl_protectmode5 where p='p1';

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/queries/clientnegative/protectmode_part2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/protectmode_part2.q b/ql/src/test/queries/clientnegative/protectmode_part2.q
deleted file mode 100644
index 3fdc036..0000000
--- a/ql/src/test/queries/clientnegative/protectmode_part2.q
+++ /dev/null
@@ -1,9 +0,0 @@
--- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode
-
-drop table tbl_protectmode6;
-
-create table tbl_protectmode6  (c1 string,c2 string) partitioned by (p string);
-alter table tbl_protectmode6 add partition (p='p1');
-LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' OVERWRITE INTO TABLE tbl_protectmode6 partition (p='p1');
-alter table tbl_protectmode6 partition (p='p1') enable offline; 
-LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' OVERWRITE INTO TABLE tbl_protectmode6 partition (p='p1');

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/queries/clientnegative/protectmode_part_no_drop.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/protectmode_part_no_drop.q b/ql/src/test/queries/clientnegative/protectmode_part_no_drop.q
deleted file mode 100644
index b4e508f..0000000
--- a/ql/src/test/queries/clientnegative/protectmode_part_no_drop.q
+++ /dev/null
@@ -1,10 +0,0 @@
--- protect mode: syntax to change protect mode works and queries to drop partitions are blocked if it is marked no drop
-
-drop table tbl_protectmode_no_drop;
-
-create table tbl_protectmode_no_drop  (c1 string,c2 string) partitioned by (p string);
-alter table tbl_protectmode_no_drop add partition (p='p1');
-alter table tbl_protectmode_no_drop partition (p='p1') enable no_drop;
-desc extended tbl_protectmode_no_drop partition (p='p1');
-
-alter table tbl_protectmode_no_drop drop partition (p='p1');

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/queries/clientnegative/protectmode_part_no_drop2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/protectmode_part_no_drop2.q b/ql/src/test/queries/clientnegative/protectmode_part_no_drop2.q
deleted file mode 100644
index e7e8c42..0000000
--- a/ql/src/test/queries/clientnegative/protectmode_part_no_drop2.q
+++ /dev/null
@@ -1,11 +0,0 @@
--- protect mode: syntax to change protect mode works and queries to drop partitions are blocked if it is marked no drop
-
-create database if not exists db1;
-use db1;
-
-create table tbl_protectmode_no_drop2  (c1 string,c2 string) partitioned by (p string);
-alter table tbl_protectmode_no_drop2 add partition (p='p1');
-alter table tbl_protectmode_no_drop2 partition (p='p1') enable no_drop;
-
-use default;
-drop table db1.tbl_protectmode_no_drop2;

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/queries/clientnegative/protectmode_tbl1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/protectmode_tbl1.q b/ql/src/test/queries/clientnegative/protectmode_tbl1.q
deleted file mode 100644
index 2361299..0000000
--- a/ql/src/test/queries/clientnegative/protectmode_tbl1.q
+++ /dev/null
@@ -1,8 +0,0 @@
--- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode
-
-drop table tbl_protectmode_1;
-
-create table tbl_protectmode_1  (col string);
-select * from tbl_protectmode_1;
-alter table tbl_protectmode_1 enable offline;
-select * from tbl_protectmode_1;

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/queries/clientnegative/protectmode_tbl2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/protectmode_tbl2.q b/ql/src/test/queries/clientnegative/protectmode_tbl2.q
deleted file mode 100644
index 05964c3..0000000
--- a/ql/src/test/queries/clientnegative/protectmode_tbl2.q
+++ /dev/null
@@ -1,12 +0,0 @@
--- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode
-
-drop table tbl_protectmode2;
-
-create table tbl_protectmode2  (col string) partitioned by (p string);
-alter table tbl_protectmode2 add partition (p='p1');
-alter table tbl_protectmode2 enable no_drop;
-alter table tbl_protectmode2 enable offline;
-alter table tbl_protectmode2 disable no_drop;
-desc extended tbl_protectmode2;
-
-select * from tbl_protectmode2 where p='p1';

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/queries/clientnegative/protectmode_tbl3.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/protectmode_tbl3.q b/ql/src/test/queries/clientnegative/protectmode_tbl3.q
deleted file mode 100644
index bbaa267..0000000
--- a/ql/src/test/queries/clientnegative/protectmode_tbl3.q
+++ /dev/null
@@ -1,10 +0,0 @@
--- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode
-
-drop table tbl_protectmode_4;
-
-create table tbl_protectmode_4  (col string);
-select col from tbl_protectmode_4;
-alter table tbl_protectmode_4 enable offline;
-desc extended tbl_protectmode_4;
-
-select col from tbl_protectmode_4;

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/queries/clientnegative/protectmode_tbl4.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/protectmode_tbl4.q b/ql/src/test/queries/clientnegative/protectmode_tbl4.q
deleted file mode 100644
index c7880de..0000000
--- a/ql/src/test/queries/clientnegative/protectmode_tbl4.q
+++ /dev/null
@@ -1,15 +0,0 @@
--- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode
-
-drop table tbl_protectmode_tbl4;
-drop table tbl_protectmode_tbl4_src;
-
-create table tbl_protectmode_tbl4_src (col string);
-
-create table tbl_protectmode_tbl4  (col string) partitioned by (p string);
-alter table tbl_protectmode_tbl4 add partition (p='p1');
-alter table tbl_protectmode_tbl4 enable no_drop;
-alter table tbl_protectmode_tbl4 enable offline;
-alter table tbl_protectmode_tbl4 disable no_drop;
-desc extended tbl_protectmode_tbl4;
-
-select col from tbl_protectmode_tbl4 where p='not_exist';

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/queries/clientnegative/protectmode_tbl5.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/protectmode_tbl5.q b/ql/src/test/queries/clientnegative/protectmode_tbl5.q
deleted file mode 100644
index cd848fd..0000000
--- a/ql/src/test/queries/clientnegative/protectmode_tbl5.q
+++ /dev/null
@@ -1,15 +0,0 @@
--- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode
-
-drop table tbl_protectmode_tbl5;
-drop table tbl_protectmode_tbl5_src;
-
-create table tbl_protectmode_tbl5_src (col string);
-
-create table tbl_protectmode_tbl5  (col string) partitioned by (p string);
-alter table tbl_protectmode_tbl5 add partition (p='p1');
-alter table tbl_protectmode_tbl5 enable no_drop;
-alter table tbl_protectmode_tbl5 enable offline;
-alter table tbl_protectmode_tbl5 disable no_drop;
-desc extended tbl_protectmode_tbl5;
-
-insert overwrite table tbl_protectmode_tbl5 partition (p='not_exist') select col from tbl_protectmode_tbl5_src;

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/queries/clientnegative/protectmode_tbl6.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/protectmode_tbl6.q b/ql/src/test/queries/clientnegative/protectmode_tbl6.q
deleted file mode 100644
index 26248cc..0000000
--- a/ql/src/test/queries/clientnegative/protectmode_tbl6.q
+++ /dev/null
@@ -1,8 +0,0 @@
--- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode
-
-drop table tbl_protectmode_tbl6;
-
-create table tbl_protectmode_tbl6 (col string);
-alter table tbl_protectmode_tbl6 enable no_drop cascade;
-
-drop table tbl_protectmode_tbl6;

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/queries/clientnegative/protectmode_tbl7.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/protectmode_tbl7.q b/ql/src/test/queries/clientnegative/protectmode_tbl7.q
deleted file mode 100644
index afff840..0000000
--- a/ql/src/test/queries/clientnegative/protectmode_tbl7.q
+++ /dev/null
@@ -1,13 +0,0 @@
--- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode
-
-drop table tbl_protectmode_tbl7;
-create table tbl_protectmode_tbl7  (col string) partitioned by (p string);
-alter table tbl_protectmode_tbl7 add partition (p='p1');
-alter table tbl_protectmode_tbl7 enable no_drop;
-
-alter table tbl_protectmode_tbl7 drop partition (p='p1');
-
-alter table tbl_protectmode_tbl7 add partition (p='p1');
-alter table tbl_protectmode_tbl7 enable no_drop cascade;
-
-alter table tbl_protectmode_tbl7 drop partition (p='p1');

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/queries/clientnegative/protectmode_tbl8.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/protectmode_tbl8.q b/ql/src/test/queries/clientnegative/protectmode_tbl8.q
deleted file mode 100644
index 809c287..0000000
--- a/ql/src/test/queries/clientnegative/protectmode_tbl8.q
+++ /dev/null
@@ -1,13 +0,0 @@
--- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode
-
-drop table tbl_protectmode_tbl8;
-create table tbl_protectmode_tbl8  (col string) partitioned by (p string);
-alter table tbl_protectmode_tbl8 add partition (p='p1');
-alter table tbl_protectmode_tbl8 enable no_drop;
-
-alter table tbl_protectmode_tbl8 drop partition (p='p1');
-
-alter table tbl_protectmode_tbl8 enable no_drop cascade;
-
-alter table tbl_protectmode_tbl8 add partition (p='p1');
-alter table tbl_protectmode_tbl8 drop partition (p='p1');

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/queries/clientnegative/protectmode_tbl_no_drop.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/protectmode_tbl_no_drop.q b/ql/src/test/queries/clientnegative/protectmode_tbl_no_drop.q
deleted file mode 100644
index a4ef2ac..0000000
--- a/ql/src/test/queries/clientnegative/protectmode_tbl_no_drop.q
+++ /dev/null
@@ -1,9 +0,0 @@
--- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode
-
-drop table tbl_protectmode__no_drop;
-
-create table tbl_protectmode__no_drop  (col string);
-select * from tbl_protectmode__no_drop;
-alter table tbl_protectmode__no_drop enable no_drop;
-desc extended tbl_protectmode__no_drop;
-drop table tbl_protectmode__no_drop;

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/queries/clientnegative/sa_fail_hook3.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/sa_fail_hook3.q b/ql/src/test/queries/clientnegative/sa_fail_hook3.q
deleted file mode 100644
index e54201c..0000000
--- a/ql/src/test/queries/clientnegative/sa_fail_hook3.q
+++ /dev/null
@@ -1,4 +0,0 @@
-create table mp2 (a string) partitioned by (b string);
-alter table mp2 add partition (b='1');
-alter table mp2 partition (b='1') enable NO_DROP;
-alter table mp2 drop partition (b='1');

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/queries/clientpositive/alter_partition_protect_mode.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/alter_partition_protect_mode.q b/ql/src/test/queries/clientpositive/alter_partition_protect_mode.q
deleted file mode 100644
index 7a1f3dd..0000000
--- a/ql/src/test/queries/clientpositive/alter_partition_protect_mode.q
+++ /dev/null
@@ -1,26 +0,0 @@
--- Create table
-create table if not exists alter_part_protect_mode(key string, value string ) partitioned by (year string, month string) stored as textfile ;
-
--- Load data
-load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_protect_mode partition (year='1996', month='10');
-load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_protect_mode partition (year='1996', month='12');
-load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_protect_mode partition (year='1995', month='09');
-load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_protect_mode partition (year='1994', month='07');
-
--- offline
-alter table alter_part_protect_mode partition (year='1996') disable offline;
-select * from alter_part_protect_mode where year = '1996';
-alter table alter_part_protect_mode partition (year='1995') enable offline;
-alter table alter_part_protect_mode partition (year='1995') disable offline;
-select * from alter_part_protect_mode where year = '1995';
-
--- no_drop
-alter table alter_part_protect_mode partition (year='1996') enable no_drop;
-alter table alter_part_protect_mode partition (year='1995') disable no_drop;
-alter table alter_part_protect_mode drop partition (year='1995');
-alter table alter_part_protect_mode partition (year='1994', month='07') disable no_drop;
-alter table alter_part_protect_mode drop partition (year='1994');
-
--- Cleanup
-alter table alter_part_protect_mode partition (year='1996') disable no_drop;
-drop table alter_part_protect_mode;

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/queries/clientpositive/drop_partitions_ignore_protection.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/drop_partitions_ignore_protection.q b/ql/src/test/queries/clientpositive/drop_partitions_ignore_protection.q
deleted file mode 100644
index e825df9..0000000
--- a/ql/src/test/queries/clientpositive/drop_partitions_ignore_protection.q
+++ /dev/null
@@ -1,10 +0,0 @@
-create table tbl_protectmode_no_drop  (c1 string,c2 string) partitioned by (p string);
-alter table tbl_protectmode_no_drop add partition (p='p1');
-alter table tbl_protectmode_no_drop partition (p='p1') enable no_drop;
-desc extended tbl_protectmode_no_drop partition (p='p1');
-
--- The partition will be dropped, even though we have enabled no_drop
--- as 'ignore protection' has been specified in the command predicate
-alter table tbl_protectmode_no_drop drop partition (p='p1') ignore protection;
-drop table tbl_protectmode_no_drop;
-

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/queries/clientpositive/protectmode.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/protectmode.q b/ql/src/test/queries/clientpositive/protectmode.q
deleted file mode 100644
index 27055fb..0000000
--- a/ql/src/test/queries/clientpositive/protectmode.q
+++ /dev/null
@@ -1,63 +0,0 @@
--- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode
-
-drop table tbl1;
-drop table tbl2;
-
-create table tbl1  (col string);
-select * from tbl1;
-select col from tbl1;
-alter table tbl1 enable offline;
-desc extended tbl1;
-alter table tbl1 disable offline;
-desc extended tbl1;
-select * from tbl1;
-select col from tbl1;
- 
-create table tbl2  (col string) partitioned by (p string);
-alter table tbl2 add partition (p='p1');
-alter table tbl2 add partition (p='p2');
-alter table tbl2 add partition (p='p3');
-alter table tbl2 drop partition (p='not_exist');
-
-select * from tbl2 where p='p1';
-select * from tbl2 where p='p2';
-
-alter table tbl2 partition (p='p1') enable offline;
-desc extended tbl2 partition (p='p1');
-
-alter table tbl2 enable offline;
-desc extended tbl2;
-
-alter table tbl2 enable no_drop;
-desc extended tbl2;
-alter table tbl2 drop partition (p='p3');
-
-alter table tbl2 disable offline;
-desc extended tbl2;
-
-alter table tbl2 disable no_drop;
-desc extended tbl2;
-
-select * from tbl2 where p='p2';
-select col from tbl2 where p='p2';
-
-alter table tbl2 partition (p='p1') disable offline;
-desc extended tbl2 partition (p='p1');
-
-select * from tbl2 where p='p1';
-select col from tbl2 where p='p1';
-
-insert overwrite table tbl1 select col from tbl2 where p='p1';
-insert overwrite table tbl1 select col from tbl1;
-
-alter table tbl2 partition (p='p1') enable no_drop;
-alter table tbl2 partition (p='p1') disable no_drop;
-
-alter table tbl2 partition (p='p2') enable no_drop;
-
-alter table tbl2 drop partition (p='p1');
-
-alter table tbl2 partition (p='p2') disable no_drop;
-
-drop table tbl1;
-drop table tbl2;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/queries/clientpositive/protectmode2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/protectmode2.q b/ql/src/test/queries/clientpositive/protectmode2.q
deleted file mode 100644
index 978b032..0000000
--- a/ql/src/test/queries/clientpositive/protectmode2.q
+++ /dev/null
@@ -1,23 +0,0 @@
-drop table tbl1;
-
-create table tbl1 (col string);
-alter table tbl1 enable no_drop cascade;
-desc extended tbl1;
-alter table tbl1 enable no_drop;
-desc extended tbl1;
-alter table tbl1 disable no_drop cascade;
-desc extended tbl1;
-alter table tbl1 disable no_drop;
-
-drop table tbl1;
-
-drop table tbl2;
-create table tbl2 (col string) partitioned by (p string);
-alter table tbl2 add partition (p='p1');
-alter table tbl2 add partition (p='p2');
-alter table tbl2 add partition (p='p3');
-alter table tbl2 enable no_drop cascade;
-desc formatted tbl2;
-alter table tbl2 disable no_drop cascade;
-desc formatted tbl2;
-drop table tbl2;

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/beelinepositive/alter_numbuckets_partitioned_table.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/beelinepositive/alter_numbuckets_partitioned_table.q.out b/ql/src/test/results/beelinepositive/alter_numbuckets_partitioned_table.q.out
index b5b089a..b0ccce5 100644
--- a/ql/src/test/results/beelinepositive/alter_numbuckets_partitioned_table.q.out
+++ b/ql/src/test/results/beelinepositive/alter_numbuckets_partitioned_table.q.out
@@ -24,7 +24,6 @@ No rows affected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/alter_numbuckets_partitioned_table.db/tst1',''
 'Table Type:         ','MANAGED_TABLE       ',''
@@ -69,7 +68,6 @@ No rows selected
 'Table:              ','tst1                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/alter_numbuckets_partitioned_table.db/tst1/ds=1',''
 'Partition Parameters:','',''
 '','numFiles            ','1                   '
@@ -117,7 +115,6 @@ No rows selected
 'Table:              ','tst1                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/alter_numbuckets_partitioned_table.db/tst1/ds=1',''
 'Partition Parameters:','',''
 '','numFiles            ','1                   '
@@ -155,7 +152,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/alter_numbuckets_partitioned_table.db/tst1',''
 'Table Type:         ','MANAGED_TABLE       ',''
@@ -203,7 +199,6 @@ No rows affected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/alter_numbuckets_partitioned_table.db/tst1',''
 'Table Type:         ','MANAGED_TABLE       ',''
@@ -251,7 +246,6 @@ No rows affected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/alter_numbuckets_partitioned_table.db/tst1',''
 'Table Type:         ','MANAGED_TABLE       ',''
@@ -299,7 +293,6 @@ No rows affected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/alter_numbuckets_partitioned_table.db/tst1',''
 'Table Type:         ','MANAGED_TABLE       ',''
@@ -347,7 +340,6 @@ No rows affected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/alter_numbuckets_partitioned_table.db/tst1',''
 'Table Type:         ','MANAGED_TABLE       ',''

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/beelinepositive/create_like.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/beelinepositive/create_like.q.out b/ql/src/test/results/beelinepositive/create_like.q.out
index 40b6cb7..df1ccc3 100644
--- a/ql/src/test/results/beelinepositive/create_like.q.out
+++ b/ql/src/test/results/beelinepositive/create_like.q.out
@@ -18,7 +18,6 @@ No rows affected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/create_like.db/table1',''
 'Table Type:         ','MANAGED_TABLE       ',''
@@ -51,7 +50,6 @@ No rows affected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/create_like.db/table2',''
 'Table Type:         ','MANAGED_TABLE       ',''
@@ -90,7 +88,6 @@ No rows affected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/create_like.db/table3',''
 'Table Type:         ','EXTERNAL_TABLE      ',''

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/beelinepositive/create_like2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/beelinepositive/create_like2.q.out b/ql/src/test/results/beelinepositive/create_like2.q.out
index 8bdb44f..ca6c69a 100644
--- a/ql/src/test/results/beelinepositive/create_like2.q.out
+++ b/ql/src/test/results/beelinepositive/create_like2.q.out
@@ -24,7 +24,6 @@ No rows affected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/create_like2.db/table2',''
 'Table Type:         ','MANAGED_TABLE       ',''

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/beelinepositive/create_like_view.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/beelinepositive/create_like_view.q.out b/ql/src/test/results/beelinepositive/create_like_view.q.out
index 80483c3..4d5ede1 100644
--- a/ql/src/test/results/beelinepositive/create_like_view.q.out
+++ b/ql/src/test/results/beelinepositive/create_like_view.q.out
@@ -28,7 +28,6 @@ No rows affected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/create_like_view.db/table1',''
 'Table Type:         ','MANAGED_TABLE       ',''
@@ -70,7 +69,6 @@ No rows affected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/create_like_view.db/table2',''
 'Table Type:         ','MANAGED_TABLE       ',''
@@ -114,7 +112,6 @@ No rows affected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/create_like_view.db/table3',''
 'Table Type:         ','EXTERNAL_TABLE      ',''
@@ -182,7 +179,6 @@ No rows affected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/create_like_view.db/table1',''
 'Table Type:         ','MANAGED_TABLE       ',''

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/beelinepositive/create_skewed_table1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/beelinepositive/create_skewed_table1.q.out b/ql/src/test/results/beelinepositive/create_skewed_table1.q.out
index 60b54a7..c887e28 100644
--- a/ql/src/test/results/beelinepositive/create_skewed_table1.q.out
+++ b/ql/src/test/results/beelinepositive/create_skewed_table1.q.out
@@ -20,7 +20,6 @@ No rows affected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/create_skewed_table1.db/list_bucket_single_2',''
 'Table Type:         ','MANAGED_TABLE       ',''
@@ -52,7 +51,6 @@ No rows affected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/create_skewed_table1.db/list_bucket_single',''
 'Table Type:         ','MANAGED_TABLE       ',''
@@ -85,7 +83,6 @@ No rows affected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/create_skewed_table1.db/list_bucket_multiple',''
 'Table Type:         ','MANAGED_TABLE       ',''

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/beelinepositive/create_view.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/beelinepositive/create_view.q.out b/ql/src/test/results/beelinepositive/create_view.q.out
index 0ec69d1..2ae4e08 100644
--- a/ql/src/test/results/beelinepositive/create_view.q.out
+++ b/ql/src/test/results/beelinepositive/create_view.q.out
@@ -170,7 +170,6 @@ SELECT * from view2 where key=18;
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Table Type:         ','VIRTUAL_VIEW        ',''
 'Table Parameters:','',''
@@ -213,7 +212,6 @@ SELECT * from view2 where key=18;
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Table Type:         ','VIRTUAL_VIEW        ',''
 'Table Parameters:','',''
@@ -253,7 +251,6 @@ SELECT * from view2 where key=18;
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Table Type:         ','VIRTUAL_VIEW        ',''
 'Table Parameters:','',''
@@ -293,7 +290,6 @@ No rows affected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Table Type:         ','VIRTUAL_VIEW        ',''
 'Table Parameters:','',''
@@ -503,7 +499,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Table Type:         ','VIRTUAL_VIEW        ',''
 'Table Parameters:','',''
@@ -560,7 +555,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Table Type:         ','VIRTUAL_VIEW        ',''
 'Table Parameters:','',''
@@ -614,7 +608,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Table Type:         ','VIRTUAL_VIEW        ',''
 'Table Parameters:','',''
@@ -664,7 +657,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Table Type:         ','VIRTUAL_VIEW        ',''
 'Table Parameters:','',''
@@ -716,7 +708,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Table Type:         ','VIRTUAL_VIEW        ',''
 'Table Parameters:','',''
@@ -770,7 +761,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Table Type:         ','VIRTUAL_VIEW        ',''
 'Table Parameters:','',''
@@ -827,7 +817,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Table Type:         ','VIRTUAL_VIEW        ',''
 'Table Parameters:','',''
@@ -920,7 +909,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Table Type:         ','VIRTUAL_VIEW        ',''
 'Table Parameters:','',''
@@ -1015,7 +1003,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Table Type:         ','VIRTUAL_VIEW        ',''
 'Table Parameters:','',''
@@ -1079,7 +1066,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Table Type:         ','VIRTUAL_VIEW        ',''
 'Table Parameters:','',''

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/beelinepositive/create_view_partitioned.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/beelinepositive/create_view_partitioned.q.out b/ql/src/test/results/beelinepositive/create_view_partitioned.q.out
index 1f0717e..9460960 100644
--- a/ql/src/test/results/beelinepositive/create_view_partitioned.q.out
+++ b/ql/src/test/results/beelinepositive/create_view_partitioned.q.out
@@ -44,7 +44,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Table Type:         ','VIRTUAL_VIEW        ',''
 'Table Parameters:','',''
@@ -186,7 +185,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Table Type:         ','VIRTUAL_VIEW        ',''
 'Table Parameters:','',''
@@ -258,7 +256,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Table Type:         ','VIRTUAL_VIEW        ',''
 'Table Parameters:','',''

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/beelinepositive/ctas.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/beelinepositive/ctas.q.out b/ql/src/test/results/beelinepositive/ctas.q.out
index 5ba3374..15e3355 100644
--- a/ql/src/test/results/beelinepositive/ctas.q.out
+++ b/ql/src/test/results/beelinepositive/ctas.q.out
@@ -144,7 +144,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/ctas.db/nzhang_ctas1',''
 'Table Type:         ','MANAGED_TABLE       ',''
@@ -299,7 +298,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/ctas.db/nzhang_ctas2',''
 'Table Type:         ','MANAGED_TABLE       ',''
@@ -455,7 +453,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/ctas.db/nzhang_ctas3',''
 'Table Type:         ','MANAGED_TABLE       ',''
@@ -520,7 +517,6 @@ No rows affected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/ctas.db/nzhang_ctas3',''
 'Table Type:         ','MANAGED_TABLE       ',''
@@ -676,7 +672,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/ctas.db/nzhang_ctas4',''
 'Table Type:         ','MANAGED_TABLE       ',''

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/beelinepositive/describe_formatted_view_partitioned.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/beelinepositive/describe_formatted_view_partitioned.q.out b/ql/src/test/results/beelinepositive/describe_formatted_view_partitioned.q.out
index 420f8df..f393f58 100644
--- a/ql/src/test/results/beelinepositive/describe_formatted_view_partitioned.q.out
+++ b/ql/src/test/results/beelinepositive/describe_formatted_view_partitioned.q.out
@@ -33,7 +33,6 @@ No rows affected
 'Table:              ','view_partitioned    ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','null                ',''
 'Partition Parameters:','',''
 '','transient_lastDdlTime','!!UNIXTIME!!          '

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/beelinepositive/describe_table.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/beelinepositive/describe_table.q.out b/ql/src/test/results/beelinepositive/describe_table.q.out
index 71d1a54..1ad5134 100644
--- a/ql/src/test/results/beelinepositive/describe_table.q.out
+++ b/ql/src/test/results/beelinepositive/describe_table.q.out
@@ -60,7 +60,6 @@ Saving all output to "!!{outputDirectory}!!/describe_table.q.raw". Enter "record
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/describe_table.db/srcpart',''
 'Table Type:         ','MANAGED_TABLE       ',''
@@ -108,7 +107,6 @@ Saving all output to "!!{outputDirectory}!!/describe_table.q.raw". Enter "record
 'Table:              ','srcpart             ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/describe_table.db/srcpart/ds=2008-04-08/hr=12',''
 'Partition Parameters:','',''
 '','numFiles            ','1                   '
@@ -157,7 +155,6 @@ No rows affected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/describe_table.db/srcpart_serdeprops',''
 'Table Type:         ','MANAGED_TABLE       ',''

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/beelinepositive/merge3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/beelinepositive/merge3.q.out b/ql/src/test/results/beelinepositive/merge3.q.out
index 5d85293..7e7d8cb 100644
--- a/ql/src/test/results/beelinepositive/merge3.q.out
+++ b/ql/src/test/results/beelinepositive/merge3.q.out
@@ -2283,7 +2283,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/merge3.db/merge_src2',''
 'Table Type:         ','MANAGED_TABLE       ',''

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/beelinepositive/part_inherit_tbl_props.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/beelinepositive/part_inherit_tbl_props.q.out b/ql/src/test/results/beelinepositive/part_inherit_tbl_props.q.out
index b436880..0b7ba0e 100644
--- a/ql/src/test/results/beelinepositive/part_inherit_tbl_props.q.out
+++ b/ql/src/test/results/beelinepositive/part_inherit_tbl_props.q.out
@@ -25,7 +25,6 @@ No rows affected
 'Table:              ','mytbl               ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/part_inherit_tbl_props.db/mytbl/c2=v1',''
 'Partition Parameters:','',''
 '','a                   ','myval               '

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/beelinepositive/part_inherit_tbl_props_empty.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/beelinepositive/part_inherit_tbl_props_empty.q.out b/ql/src/test/results/beelinepositive/part_inherit_tbl_props_empty.q.out
index 2010d13..fa45c0c 100644
--- a/ql/src/test/results/beelinepositive/part_inherit_tbl_props_empty.q.out
+++ b/ql/src/test/results/beelinepositive/part_inherit_tbl_props_empty.q.out
@@ -23,7 +23,6 @@ No rows affected
 'Table:              ','mytbl               ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/part_inherit_tbl_props_empty.db/mytbl/c2=v1',''
 'Partition Parameters:','',''
 '','transient_lastDdlTime','!!UNIXTIME!!          '

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/beelinepositive/part_inherit_tbl_props_with_star.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/beelinepositive/part_inherit_tbl_props_with_star.q.out b/ql/src/test/results/beelinepositive/part_inherit_tbl_props_with_star.q.out
index 27c58e0..b6d964e 100644
--- a/ql/src/test/results/beelinepositive/part_inherit_tbl_props_with_star.q.out
+++ b/ql/src/test/results/beelinepositive/part_inherit_tbl_props_with_star.q.out
@@ -25,7 +25,6 @@ No rows affected
 'Table:              ','mytbl               ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/part_inherit_tbl_props_with_star.db/mytbl/c2=v1',''
 'Partition Parameters:','',''
 '','a                   ','myval               '

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/beelinepositive/protectmode2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/beelinepositive/protectmode2.q.out b/ql/src/test/results/beelinepositive/protectmode2.q.out
index 9995420..acaa26b 100644
--- a/ql/src/test/results/beelinepositive/protectmode2.q.out
+++ b/ql/src/test/results/beelinepositive/protectmode2.q.out
@@ -63,7 +63,6 @@ No rows affected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','NO_DROP_CASCADE     ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/protectmode2.db/tbl2',''
 'Table Type:         ','MANAGED_TABLE       ',''
@@ -102,7 +101,6 @@ No rows affected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/protectmode2.db/tbl2',''
 'Table Type:         ','MANAGED_TABLE       ',''

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/beelinepositive/stats1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/beelinepositive/stats1.q.out b/ql/src/test/results/beelinepositive/stats1.q.out
index a6d10df..91a9f5c 100644
--- a/ql/src/test/results/beelinepositive/stats1.q.out
+++ b/ql/src/test/results/beelinepositive/stats1.q.out
@@ -185,7 +185,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats1.db/tmptable',''
 'Table Type:         ','MANAGED_TABLE       ',''
@@ -226,7 +225,6 @@ No rows affected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats1.db/tmptable',''
 'Table Type:         ','MANAGED_TABLE       ',''

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/beelinepositive/stats10.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/beelinepositive/stats10.q.out b/ql/src/test/results/beelinepositive/stats10.q.out
index 803a897..a7b38e7 100644
--- a/ql/src/test/results/beelinepositive/stats10.q.out
+++ b/ql/src/test/results/beelinepositive/stats10.q.out
@@ -395,7 +395,6 @@ No rows selected
 'Table:              ','bucket3_1           ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats10.db/bucket3_1/ds=1',''
 'Partition Parameters:','',''
 '','numFiles            ','2                   '
@@ -433,7 +432,6 @@ No rows selected
 'Table:              ','bucket3_1           ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats10.db/bucket3_1/ds=2',''
 'Partition Parameters:','',''
 '','numFiles            ','2                   '
@@ -470,7 +468,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats10.db/bucket3_1',''
 'Table Type:         ','MANAGED_TABLE       ',''

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/beelinepositive/stats11.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/beelinepositive/stats11.q.out b/ql/src/test/results/beelinepositive/stats11.q.out
index e86aaf9..51bd85b 100644
--- a/ql/src/test/results/beelinepositive/stats11.q.out
+++ b/ql/src/test/results/beelinepositive/stats11.q.out
@@ -69,7 +69,6 @@ No rows affected
 'Table:              ','srcbucket_mapjoin_part',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats11.db/srcbucket_mapjoin_part/ds=2008-04-08',''
 'Partition Parameters:','',''
 '','numFiles            ','1                   '
@@ -109,7 +108,6 @@ No rows affected
 'Table:              ','srcbucket_mapjoin_part',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats11.db/srcbucket_mapjoin_part/ds=2008-04-08',''
 'Partition Parameters:','',''
 '','numFiles            ','2                   '
@@ -149,7 +147,6 @@ No rows affected
 'Table:              ','srcbucket_mapjoin_part',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats11.db/srcbucket_mapjoin_part/ds=2008-04-08',''
 'Partition Parameters:','',''
 '','numFiles            ','3                   '
@@ -189,7 +186,6 @@ No rows affected
 'Table:              ','srcbucket_mapjoin_part',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats11.db/srcbucket_mapjoin_part/ds=2008-04-08',''
 'Partition Parameters:','',''
 '','numFiles            ','4                   '

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/beelinepositive/stats12.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/beelinepositive/stats12.q.out b/ql/src/test/results/beelinepositive/stats12.q.out
index f6e1634..1774243 100644
--- a/ql/src/test/results/beelinepositive/stats12.q.out
+++ b/ql/src/test/results/beelinepositive/stats12.q.out
@@ -151,7 +151,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats12.db/analyze_srcpart',''
 'Table Type:         ','MANAGED_TABLE       ',''
@@ -193,7 +192,6 @@ No rows selected
 'Table:              ','analyze_srcpart     ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats12.db/analyze_srcpart/ds=2008-04-08/hr=11',''
 'Partition Parameters:','',''
 '','numFiles            ','1                   '
@@ -232,7 +230,6 @@ No rows selected
 'Table:              ','analyze_srcpart     ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats12.db/analyze_srcpart/ds=2008-04-08/hr=12',''
 'Partition Parameters:','',''
 '','numFiles            ','1                   '
@@ -271,7 +268,6 @@ No rows selected
 'Table:              ','analyze_srcpart     ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats12.db/analyze_srcpart/ds=2008-04-09/hr=11',''
 'Partition Parameters:','',''
 '','transient_lastDdlTime','!!UNIXTIME!!          '
@@ -306,7 +302,6 @@ No rows selected
 'Table:              ','analyze_srcpart     ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats12.db/analyze_srcpart/ds=2008-04-09/hr=12',''
 'Partition Parameters:','',''
 '','transient_lastDdlTime','!!UNIXTIME!!          '

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/beelinepositive/stats13.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/beelinepositive/stats13.q.out b/ql/src/test/results/beelinepositive/stats13.q.out
index 9f2142e..17f3ef5 100644
--- a/ql/src/test/results/beelinepositive/stats13.q.out
+++ b/ql/src/test/results/beelinepositive/stats13.q.out
@@ -109,7 +109,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats13.db/analyze_srcpart',''
 'Table Type:         ','MANAGED_TABLE       ',''
@@ -151,7 +150,6 @@ No rows selected
 'Table:              ','analyze_srcpart     ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats13.db/analyze_srcpart/ds=2008-04-08/hr=11',''
 'Partition Parameters:','',''
 '','numFiles            ','1                   '
@@ -190,7 +188,6 @@ No rows selected
 'Table:              ','analyze_srcpart     ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats13.db/analyze_srcpart/ds=2008-04-08/hr=12',''
 'Partition Parameters:','',''
 '','transient_lastDdlTime','!!UNIXTIME!!          '
@@ -225,7 +222,6 @@ No rows selected
 'Table:              ','analyze_srcpart     ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats13.db/analyze_srcpart/ds=2008-04-09/hr=11',''
 'Partition Parameters:','',''
 '','transient_lastDdlTime','!!UNIXTIME!!          '
@@ -260,7 +256,6 @@ No rows selected
 'Table:              ','analyze_srcpart     ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats13.db/analyze_srcpart/ds=2008-04-09/hr=12',''
 'Partition Parameters:','',''
 '','transient_lastDdlTime','!!UNIXTIME!!          '
@@ -298,7 +293,6 @@ No rows affected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats13.db/analyze_srcpart2',''
 'Table Type:         ','MANAGED_TABLE       ',''

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/beelinepositive/stats14.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/beelinepositive/stats14.q.out b/ql/src/test/results/beelinepositive/stats14.q.out
index e9df282..2d10772 100644
--- a/ql/src/test/results/beelinepositive/stats14.q.out
+++ b/ql/src/test/results/beelinepositive/stats14.q.out
@@ -23,7 +23,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats14.db/stats_src',''
 'Table Type:         ','MANAGED_TABLE       ',''
@@ -86,7 +85,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats14.db/stats_part',''
 'Table Type:         ','MANAGED_TABLE       ',''
@@ -128,7 +126,6 @@ No rows selected
 'Table:              ','stats_part          ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats14.db/stats_part/ds=2010-04-08/hr=11',''
 'Partition Parameters:','',''
 '','numFiles            ','1                   '
@@ -167,7 +164,6 @@ No rows selected
 'Table:              ','stats_part          ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats14.db/stats_part/ds=2010-04-08/hr=12',''
 'Partition Parameters:','',''
 '','numFiles            ','1                   '
@@ -209,7 +205,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats14.db/stats_part',''
 'Table Type:         ','MANAGED_TABLE       ',''

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/beelinepositive/stats15.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/beelinepositive/stats15.q.out b/ql/src/test/results/beelinepositive/stats15.q.out
index 9be8852..ceaf500 100644
--- a/ql/src/test/results/beelinepositive/stats15.q.out
+++ b/ql/src/test/results/beelinepositive/stats15.q.out
@@ -25,7 +25,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats15.db/stats_src',''
 'Table Type:         ','MANAGED_TABLE       ',''
@@ -88,7 +87,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats15.db/stats_part',''
 'Table Type:         ','MANAGED_TABLE       ',''
@@ -130,7 +128,6 @@ No rows selected
 'Table:              ','stats_part          ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats15.db/stats_part/ds=2010-04-08/hr=11',''
 'Partition Parameters:','',''
 '','numFiles            ','1                   '
@@ -169,7 +166,6 @@ No rows selected
 'Table:              ','stats_part          ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats15.db/stats_part/ds=2010-04-08/hr=12',''
 'Partition Parameters:','',''
 '','numFiles            ','1                   '
@@ -211,7 +207,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats15.db/stats_part',''
 'Table Type:         ','MANAGED_TABLE       ',''

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/beelinepositive/stats16.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/beelinepositive/stats16.q.out b/ql/src/test/results/beelinepositive/stats16.q.out
index ea8593e..333b8d3 100644
--- a/ql/src/test/results/beelinepositive/stats16.q.out
+++ b/ql/src/test/results/beelinepositive/stats16.q.out
@@ -20,7 +20,6 @@ No rows affected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats16.db/stats16',''
 'Table Type:         ','MANAGED_TABLE       ',''
@@ -57,7 +56,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats16.db/stats16',''
 'Table Type:         ','MANAGED_TABLE       ',''

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/beelinepositive/stats18.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/beelinepositive/stats18.q.out b/ql/src/test/results/beelinepositive/stats18.q.out
index e504089..61867d4 100644
--- a/ql/src/test/results/beelinepositive/stats18.q.out
+++ b/ql/src/test/results/beelinepositive/stats18.q.out
@@ -40,7 +40,6 @@ No rows selected
 'Table:              ','stats_part          ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats18.db/stats_part/ds=2010-04-08/hr=13',''
 'Partition Parameters:','',''
 '','numFiles            ','1                   '
@@ -83,7 +82,6 @@ No rows affected
 'Table:              ','stats_part          ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats18.db/stats_part/ds=2010-04-08/hr=13',''
 'Partition Parameters:','',''
 '','numFiles            ','2                   '

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/beelinepositive/stats2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/beelinepositive/stats2.q.out b/ql/src/test/results/beelinepositive/stats2.q.out
index c958b06..82b338d 100644
--- a/ql/src/test/results/beelinepositive/stats2.q.out
+++ b/ql/src/test/results/beelinepositive/stats2.q.out
@@ -90,7 +90,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats2.db/analyze_t1',''
 'Table Type:         ','MANAGED_TABLE       ',''
@@ -154,7 +153,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats2.db/analyze_t1',''
 'Table Type:         ','MANAGED_TABLE       ',''

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/beelinepositive/stats3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/beelinepositive/stats3.q.out b/ql/src/test/results/beelinepositive/stats3.q.out
index f0680b4..d57f6b9 100644
--- a/ql/src/test/results/beelinepositive/stats3.q.out
+++ b/ql/src/test/results/beelinepositive/stats3.q.out
@@ -72,7 +72,6 @@ No rows affected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats3.db/hive_test_src',''
 'Table Type:         ','MANAGED_TABLE       ',''
@@ -161,7 +160,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats3.db/hive_test_dst',''
 'Table Type:         ','MANAGED_TABLE       ',''

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/beelinepositive/stats4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/beelinepositive/stats4.q.out b/ql/src/test/results/beelinepositive/stats4.q.out
index 421b9a7..d5be03d 100644
--- a/ql/src/test/results/beelinepositive/stats4.q.out
+++ b/ql/src/test/results/beelinepositive/stats4.q.out
@@ -2266,7 +2266,6 @@ No rows selected
 'Table:              ','nzhang_part1        ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats4.db/nzhang_part1/ds=2008-04-08/hr=11',''
 'Partition Parameters:','',''
 '','numFiles            ','1                   '
@@ -2305,7 +2304,6 @@ No rows selected
 'Table:              ','nzhang_part1        ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats4.db/nzhang_part1/ds=2008-04-08/hr=12',''
 'Partition Parameters:','',''
 '','numFiles            ','1                   '
@@ -2344,7 +2342,6 @@ No rows selected
 'Table:              ','nzhang_part2        ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats4.db/nzhang_part2/ds=2008-12-31/hr=11',''
 'Partition Parameters:','',''
 '','numFiles            ','1                   '
@@ -2383,7 +2380,6 @@ No rows selected
 'Table:              ','nzhang_part2        ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats4.db/nzhang_part2/ds=2008-12-31/hr=12',''
 'Partition Parameters:','',''
 '','numFiles            ','1                   '
@@ -2422,7 +2418,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats4.db/nzhang_part1',''
 'Table Type:         ','MANAGED_TABLE       ',''
@@ -2463,7 +2458,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats4.db/nzhang_part2',''
 'Table Type:         ','MANAGED_TABLE       ',''

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/beelinepositive/stats5.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/beelinepositive/stats5.q.out b/ql/src/test/results/beelinepositive/stats5.q.out
index 4339503..40c8c5b 100644
--- a/ql/src/test/results/beelinepositive/stats5.q.out
+++ b/ql/src/test/results/beelinepositive/stats5.q.out
@@ -48,7 +48,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats5.db/analyze_src',''
 'Table Type:         ','MANAGED_TABLE       ',''

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/beelinepositive/stats6.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/beelinepositive/stats6.q.out b/ql/src/test/results/beelinepositive/stats6.q.out
index 3d2bdcb..98a19c6 100644
--- a/ql/src/test/results/beelinepositive/stats6.q.out
+++ b/ql/src/test/results/beelinepositive/stats6.q.out
@@ -41,7 +41,6 @@ No rows selected
 'Table:              ','analyze_srcpart     ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats6.db/analyze_srcpart/ds=2008-04-08/hr=11',''
 'Partition Parameters:','',''
 '','numFiles            ','1                   '
@@ -80,7 +79,6 @@ No rows selected
 'Table:              ','analyze_srcpart     ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats6.db/analyze_srcpart/ds=2008-04-08/hr=12',''
 'Partition Parameters:','',''
 '','numFiles            ','1                   '
@@ -119,7 +117,6 @@ No rows selected
 'Table:              ','analyze_srcpart     ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats6.db/analyze_srcpart/ds=2008-04-09/hr=11',''
 'Partition Parameters:','',''
 '','transient_lastDdlTime','!!UNIXTIME!!          '
@@ -154,7 +151,6 @@ No rows selected
 'Table:              ','analyze_srcpart     ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats6.db/analyze_srcpart/ds=2008-04-09/hr=12',''
 'Partition Parameters:','',''
 '','transient_lastDdlTime','!!UNIXTIME!!          '
@@ -189,7 +185,6 @@ No rows selected
 'Owner:              ','!!{user.name}!!                ',''
 'CreateTime:         ','!!TIMESTAMP!!',''
 'LastAccessTime:     ','UNKNOWN             ',''
-'Protect Mode:       ','None                ',''
 'Retention:          ','0                   ',''
 'Location:           ','!!{hive.metastore.warehouse.dir}!!/stats6.db/analyze_srcpart',''
 'Table Type:         ','MANAGED_TABLE       ',''


[26/50] [abbrv] hive git commit: HIVE-11137. In DateWritable remove use of LazyBinaryUtils.

Posted by ga...@apache.org.
HIVE-11137. In DateWritable remove use of LazyBinaryUtils.


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

Branch: refs/heads/hbase-metastore
Commit: 7788968ce7baced7b6cbd575317d7413fc75d143
Parents: b6f48cb
Author: Owen O'Malley <om...@apache.org>
Authored: Sun Jun 28 17:04:49 2015 -0700
Committer: Owen O'Malley <om...@apache.org>
Committed: Fri Jul 17 09:51:52 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hive/serde2/io/DateWritable.java    | 14 --------------
 .../hadoop/hive/serde2/lazybinary/LazyBinaryDate.java |  4 +++-
 .../hive/serde2/lazybinary/LazyBinarySerDe.java       | 11 ++++++++---
 3 files changed, 11 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/7788968c/serde/src/java/org/apache/hadoop/hive/serde2/io/DateWritable.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/io/DateWritable.java b/serde/src/java/org/apache/hadoop/hive/serde2/io/DateWritable.java
index 6cedf4c..e69351f 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/io/DateWritable.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/io/DateWritable.java
@@ -25,9 +25,6 @@ import java.util.Calendar;
 import java.util.TimeZone;
 import java.util.concurrent.TimeUnit;
 
-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.io.WritableComparable;
 import org.apache.hadoop.io.WritableUtils;
 
@@ -146,17 +143,6 @@ public class DateWritable implements WritableComparable<DateWritable> {
     return millisToDays(millisLocal);
   }
 
-  public void setFromBytes(byte[] bytes, int offset, int length, VInt vInt) {
-    LazyBinaryUtils.readVInt(bytes, offset, vInt);
-    assert (length == vInt.length);
-    set(vInt.value);
-  }
-
-  public void writeToByteStream(RandomAccessOutput byteStream) {
-    LazyBinaryUtils.writeVInt(byteStream, getDays());
-  }
-
-
   @Override
   public void readFields(DataInput in) throws IOException {
     daysSinceEpoch = WritableUtils.readVInt(in);

http://git-wip-us.apache.org/repos/asf/hive/blob/7788968c/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryDate.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryDate.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryDate.java
index d0c2504..4200e26 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryDate.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryDate.java
@@ -55,6 +55,8 @@ public class LazyBinaryDate extends
    */
   @Override
   public void init(ByteArrayRef bytes, int start, int length) {
-    data.setFromBytes(bytes.getData(), start, length, vInt);
+    LazyBinaryUtils.readVInt(bytes.getData(), start, vInt);
+    assert (length == vInt.length);
+    data.set(vInt.value);
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/7788968c/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 3d14fbe..a5dc5d8 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
@@ -250,9 +250,9 @@ public class LazyBinarySerDe extends AbstractSerDe {
    *
    * @param byteStream
    *          the byte stream storing the serialization data
-   * @param obj
+   * @param fieldData
    *          the struct object to serialize
-   * @param objInspector
+   * @param fieldOis
    *          the struct object inspector
    * @param warnedOnceNullMapKey a boolean indicating whether a warning
    *          has been issued once already when encountering null map keys
@@ -311,6 +311,11 @@ public class LazyBinarySerDe extends AbstractSerDe {
     public boolean value;
   }
 
+  private static void writeDateToByteStream(RandomAccessOutput byteStream,
+                                            DateWritable date) {
+    LazyBinaryUtils.writeVInt(byteStream, date.getDays());
+  }
+
   /**
    * A recursive function that serialize an object to a byte buffer based on its
    * object inspector.
@@ -422,7 +427,7 @@ public class LazyBinarySerDe extends AbstractSerDe {
 
       case DATE: {
         DateWritable d = ((DateObjectInspector) poi).getPrimitiveWritableObject(obj);
-        d.writeToByteStream(byteStream);
+        writeDateToByteStream(byteStream, d);
         return;
       }
       case TIMESTAMP: {


[03/50] [abbrv] hive git commit: HIVE-11224 : AggregateStatsCache triggers java.util.ConcurrentModificationException under some conditions (Pengcheng Xiong via Thejas Nair)

Posted by ga...@apache.org.
HIVE-11224 :  AggregateStatsCache triggers java.util.ConcurrentModificationException under some conditions (Pengcheng Xiong via Thejas Nair)


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

Branch: refs/heads/hbase-metastore
Commit: 90a2cf9e87b22d9f568701dc53c8f8ffbe520fdb
Parents: a65bcbd
Author: Pengcheng Xiong <px...@hortonworks.com>
Authored: Tue Jul 14 10:46:30 2015 -0700
Committer: Thejas Nair <th...@hortonworks.com>
Committed: Tue Jul 14 10:46:30 2015 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/metastore/AggregateStatsCache.java   | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/90a2cf9e/metastore/src/java/org/apache/hadoop/hive/metastore/AggregateStatsCache.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/AggregateStatsCache.java b/metastore/src/java/org/apache/hadoop/hive/metastore/AggregateStatsCache.java
index 44106f5..65e2c65 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/AggregateStatsCache.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/AggregateStatsCache.java
@@ -241,7 +241,8 @@ public class AggregateStatsCache {
     // We'll count misses as we iterate
     int maxMisses = (int) maxVariance * numPartsRequested;
     for (String partName : partNames) {
-      for (Map.Entry<AggrColStats, MatchStats> entry : candidateMatchStats.entrySet()) {
+      for (Iterator<Map.Entry<AggrColStats, MatchStats>> iterator = candidateMatchStats.entrySet().iterator(); iterator.hasNext();) {
+        Map.Entry<AggrColStats, MatchStats> entry = iterator.next();
         AggrColStats candidate = entry.getKey();
         matchStats = entry.getValue();
         if (candidate.getBloomFilter().test(partName.getBytes())) {
@@ -252,7 +253,7 @@ public class AggregateStatsCache {
         // 2nd pass at removing invalid candidates
         // If misses so far exceed max tolerable misses
         if (matchStats.misses > maxMisses) {
-          candidateMatchStats.remove(candidate);
+          iterator.remove();
           continue;
         }
         // Check if this is the best match so far


[29/50] [abbrv] hive git commit: HIVE-11282: CBO (Calcite Return Path): Inferring Hive type char/varchar of length zero which is not allowed (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by ga...@apache.org.
HIVE-11282: CBO (Calcite Return Path): Inferring Hive type char/varchar of length zero which is not allowed (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/hbase-metastore
Commit: 7fe23aa1c6a6abd96717b44790033b49f13e550f
Parents: 2b1f03e
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Fri Jul 17 20:08:14 2015 +0100
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Fri Jul 17 20:08:14 2015 +0100

----------------------------------------------------------------------
 .../calcite/translator/ExprNodeConverter.java   | 24 +++++++++++++++-----
 1 file changed, 18 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/7fe23aa1/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ExprNodeConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ExprNodeConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ExprNodeConverter.java
index 4f0db03..955aa91 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ExprNodeConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ExprNodeConverter.java
@@ -224,12 +224,24 @@ public class ExprNodeConverter extends RexVisitorImpl<ExprNodeDesc> {
     case DECIMAL:
       return new ExprNodeConstantDesc(TypeInfoFactory.getDecimalTypeInfo(lType.getPrecision(),
           lType.getScale()), literal.getValue3());
-    case VARCHAR:
-      return new ExprNodeConstantDesc(TypeInfoFactory.getVarcharTypeInfo(lType.getPrecision()),
-          new HiveVarchar((String) literal.getValue3(), lType.getPrecision()));
-    case CHAR:
-      return new ExprNodeConstantDesc(TypeInfoFactory.getCharTypeInfo(lType.getPrecision()),
-          new HiveChar((String) literal.getValue3(), lType.getPrecision()));
+    case VARCHAR: {
+      int varcharLength = lType.getPrecision();
+      // If we cannot use Varchar due to type length restrictions, we use String
+      if (varcharLength < 1 || varcharLength > HiveVarchar.MAX_VARCHAR_LENGTH) {
+        return new ExprNodeConstantDesc(TypeInfoFactory.stringTypeInfo, literal.getValue3());
+      }
+      return new ExprNodeConstantDesc(TypeInfoFactory.getVarcharTypeInfo(varcharLength),
+          new HiveVarchar((String) literal.getValue3(), varcharLength));
+    }
+    case CHAR: {
+      int charLength = lType.getPrecision();
+      // If we cannot use Char due to type length restrictions, we use String
+      if (charLength < 1 || charLength > HiveChar.MAX_CHAR_LENGTH) {
+        return new ExprNodeConstantDesc(TypeInfoFactory.stringTypeInfo, literal.getValue3());
+      }
+      return new ExprNodeConstantDesc(TypeInfoFactory.getCharTypeInfo(charLength),
+          new HiveChar((String) literal.getValue3(), charLength));
+    }
     case INTERVAL_YEAR_MONTH: {
       BigDecimal monthsBd = (BigDecimal) literal.getValue();
       return new ExprNodeConstantDesc(TypeInfoFactory.intervalYearMonthTypeInfo,


[33/50] [abbrv] hive git commit: HIVE-11284: Fix cbo_rp_join0 failure on master (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by ga...@apache.org.
HIVE-11284: Fix cbo_rp_join0 failure on master (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/hbase-metastore
Commit: a5cc034bfa5e0ef7ce17f537abca3b35b23ecd16
Parents: 7b17df1
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Fri Jul 17 18:35:14 2015 +0100
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Sat Jul 18 11:14:13 2015 +0100

----------------------------------------------------------------------
 .../rules/HiveJoinProjectTransposeRule.java     | 223 +++++++++++++++++++
 .../hadoop/hive/ql/parse/CalcitePlanner.java    |   1 +
 .../results/clientpositive/cbo_rp_join0.q.out   |   4 +-
 3 files changed, 226 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/a5cc034b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinProjectTransposeRule.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinProjectTransposeRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinProjectTransposeRule.java
index 40bf043..fd8f5cb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinProjectTransposeRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinProjectTransposeRule.java
@@ -17,10 +17,27 @@
  */
 package org.apache.hadoop.hive.ql.optimizer.calcite.rules;
 
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.Project;
 import org.apache.calcite.rel.core.RelFactories.ProjectFactory;
 import org.apache.calcite.rel.rules.JoinProjectTransposeRule;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexProgramBuilder;
+import org.apache.calcite.util.Pair;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveJoin;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject;
 
@@ -50,11 +67,217 @@ public class HiveJoinProjectTransposeRule extends JoinProjectTransposeRule {
           "JoinProjectTransposeRule(Other-Project)",
           HiveProject.DEFAULT_PROJECT_FACTORY);
 
+  private final ProjectFactory projectFactory;
+
 
   private HiveJoinProjectTransposeRule(
       RelOptRuleOperand operand,
       String description, ProjectFactory pFactory) {
     super(operand, description, pFactory);
+    this.projectFactory = pFactory;
+  }
+
+  @Override
+  public void onMatch(RelOptRuleCall call) {
+    Join joinRel = call.rel(0);
+    JoinRelType joinType = joinRel.getJoinType();
+
+    Project leftProj;
+    Project rightProj;
+    RelNode leftJoinChild;
+    RelNode rightJoinChild;
+
+    // see if at least one input's projection doesn't generate nulls
+    if (hasLeftChild(call)) {
+      leftProj = call.rel(1);
+      leftJoinChild = getProjectChild(call, leftProj, true);
+    } else {
+      leftProj = null;
+      leftJoinChild = call.rel(1);
+    }
+    if (hasRightChild(call)) {
+      rightProj = getRightChild(call);
+      rightJoinChild = getProjectChild(call, rightProj, false);
+    } else {
+      rightProj = null;
+      rightJoinChild = joinRel.getRight();
+    }
+    if ((leftProj == null) && (rightProj == null)) {
+      return;
+    }
+
+    // Construct two RexPrograms and combine them.  The bottom program
+    // is a join of the projection expressions from the left and/or
+    // right projects that feed into the join.  The top program contains
+    // the join condition.
+
+    // Create a row type representing a concatenation of the inputs
+    // underneath the projects that feed into the join.  This is the input
+    // into the bottom RexProgram.  Note that the join type is an inner
+    // join because the inputs haven't actually been joined yet.
+    RelDataType joinChildrenRowType =
+        Join.deriveJoinRowType(
+            leftJoinChild.getRowType(),
+            rightJoinChild.getRowType(),
+            JoinRelType.INNER,
+            joinRel.getCluster().getTypeFactory(),
+            null,
+            Collections.<RelDataTypeField>emptyList());
+
+    // Create projection expressions, combining the projection expressions
+    // from the projects that feed into the join.  For the RHS projection
+    // expressions, shift them to the right by the number of fields on
+    // the LHS.  If the join input was not a projection, simply create
+    // references to the inputs.
+    int nProjExprs = joinRel.getRowType().getFieldCount();
+    List<Pair<RexNode, String>> projects =
+        new ArrayList<Pair<RexNode, String>>();
+    RexBuilder rexBuilder = joinRel.getCluster().getRexBuilder();
+
+    createProjectExprs(
+        leftProj,
+        leftJoinChild,
+        0,
+        rexBuilder,
+        joinChildrenRowType.getFieldList(),
+        projects);
+
+    List<RelDataTypeField> leftFields =
+        leftJoinChild.getRowType().getFieldList();
+    int nFieldsLeft = leftFields.size();
+    createProjectExprs(
+        rightProj,
+        rightJoinChild,
+        nFieldsLeft,
+        rexBuilder,
+        joinChildrenRowType.getFieldList(),
+        projects);
+
+    List<RelDataType> projTypes = new ArrayList<RelDataType>();
+    for (int i = 0; i < nProjExprs; i++) {
+      projTypes.add(projects.get(i).left.getType());
+    }
+    RelDataType projRowType =
+        rexBuilder.getTypeFactory().createStructType(
+            projTypes,
+            Pair.right(projects));
+
+    // create the RexPrograms and merge them
+    RexProgram bottomProgram =
+        RexProgram.create(
+            joinChildrenRowType,
+            Pair.left(projects),
+            null,
+            projRowType,
+            rexBuilder);
+    RexProgramBuilder topProgramBuilder =
+        new RexProgramBuilder(
+            projRowType,
+            rexBuilder);
+    topProgramBuilder.addIdentity();
+    topProgramBuilder.addCondition(joinRel.getCondition());
+    RexProgram topProgram = topProgramBuilder.getProgram();
+    RexProgram mergedProgram =
+        RexProgramBuilder.mergePrograms(
+            topProgram,
+            bottomProgram,
+            rexBuilder);
+
+    // expand out the join condition and construct a new LogicalJoin that
+    // directly references the join children without the intervening
+    // ProjectRels
+    RexNode newCondition =
+        mergedProgram.expandLocalRef(
+            mergedProgram.getCondition());
+    Join newJoinRel =
+        joinRel.copy(joinRel.getTraitSet(), newCondition,
+            leftJoinChild, rightJoinChild, joinRel.getJoinType(),
+            joinRel.isSemiJoinDone());
+
+    // expand out the new projection expressions; if the join is an
+    // outer join, modify the expressions to reference the join output
+    List<RexNode> newProjExprs = new ArrayList<RexNode>();
+    List<RexLocalRef> projList = mergedProgram.getProjectList();
+    List<RelDataTypeField> newJoinFields =
+        newJoinRel.getRowType().getFieldList();
+    int nJoinFields = newJoinFields.size();
+    int[] adjustments = new int[nJoinFields];
+    for (int i = 0; i < nProjExprs; i++) {
+      RexNode newExpr = mergedProgram.expandLocalRef(projList.get(i));
+      if (joinType != JoinRelType.INNER) {
+        newExpr =
+            newExpr.accept(
+                new RelOptUtil.RexInputConverter(
+                    rexBuilder,
+                    joinChildrenRowType.getFieldList(),
+                    newJoinFields,
+                    adjustments));
+      }
+      newProjExprs.add(newExpr);
+    }
+
+    // finally, create the projection on top of the join
+    RelNode newProjRel = projectFactory.createProject(newJoinRel, newProjExprs,
+        joinRel.getRowType().getFieldNames());
+
+    call.transformTo(newProjRel);
   }
 
+  /**
+   * Creates projection expressions corresponding to one of the inputs into
+   * the join
+   *
+   * @param projRel            the projection input into the join (if it exists)
+   * @param joinChild          the child of the projection input (if there is a
+   *                           projection); otherwise, this is the join input
+   * @param adjustmentAmount   the amount the expressions need to be shifted by
+   * @param rexBuilder         rex builder
+   * @param joinChildrenFields concatenation of the fields from the left and
+   *                           right join inputs (once the projections have been
+   *                           removed)
+   * @param projects           Projection expressions &amp; names to be created
+   */
+  private void createProjectExprs(
+      Project projRel,
+      RelNode joinChild,
+      int adjustmentAmount,
+      RexBuilder rexBuilder,
+      List<RelDataTypeField> joinChildrenFields,
+      List<Pair<RexNode, String>> projects) {
+    List<RelDataTypeField> childFields =
+        joinChild.getRowType().getFieldList();
+    if (projRel != null) {
+      List<Pair<RexNode, String>> namedProjects =
+          projRel.getNamedProjects();
+      int nChildFields = childFields.size();
+      int[] adjustments = new int[nChildFields];
+      for (int i = 0; i < nChildFields; i++) {
+        adjustments[i] = adjustmentAmount;
+      }
+      for (Pair<RexNode, String> pair : namedProjects) {
+        RexNode e = pair.left;
+        if (adjustmentAmount != 0) {
+          // shift the references by the adjustment amount
+          e = e.accept(
+              new RelOptUtil.RexInputConverter(
+                  rexBuilder,
+                  childFields,
+                  joinChildrenFields,
+                  adjustments));
+        }
+        projects.add(Pair.of(e, pair.right));
+      }
+    } else {
+      // no projection; just create references to the inputs
+      for (int i = 0; i < childFields.size(); i++) {
+        final RelDataTypeField field = childFields.get(i);
+        projects.add(
+            Pair.of(
+                (RexNode) rexBuilder.makeInputRef(
+                    field.getType(),
+                    i + adjustmentAmount),
+                field.getName()));
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/a5cc034b/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 1ea236b..5b469e3 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
@@ -876,6 +876,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
         // 6.1. Merge join into multijoin operators (if possible)
         calciteOptimizedPlan = hepPlan(calciteOptimizedPlan, true, mdProvider.getMetadataProvider(),
                 HepMatchOrder.BOTTOM_UP, HiveJoinProjectTransposeRule.BOTH_PROJECT,
+                HiveJoinProjectTransposeRule.LEFT_PROJECT, HiveJoinProjectTransposeRule.RIGHT_PROJECT,
                 HiveJoinToMultiJoinRule.INSTANCE, HiveProjectMergeRule.INSTANCE);
         // The previous rules can pull up projections through join operators,
         // thus we run the field trimmer again to push them back down

http://git-wip-us.apache.org/repos/asf/hive/blob/a5cc034b/ql/src/test/results/clientpositive/cbo_rp_join0.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/cbo_rp_join0.q.out b/ql/src/test/results/clientpositive/cbo_rp_join0.q.out
index 93fed08..a8bcc90 100644
--- a/ql/src/test/results/clientpositive/cbo_rp_join0.q.out
+++ b/ql/src/test/results/clientpositive/cbo_rp_join0.q.out
@@ -738,10 +738,10 @@ STAGE PLANS:
             1 key (type: string)
             2 key (type: string)
             3 key (type: string)
-          outputColumnNames: key, c_int, key0, c_int0, key1, c_int1
+          outputColumnNames: key, c_int, key0, c_int0, key1, c_int2
           Statistics: Num rows: 1458 Data size: 389286 Basic stats: COMPLETE Column stats: COMPLETE
           Select Operator
-            expressions: key (type: string), c_int (type: int), key0 (type: string), c_int0 (type: int), key1 (type: string), c_int1 (type: int)
+            expressions: key (type: string), c_int (type: int), key0 (type: string), c_int0 (type: int), key1 (type: string), c_int2 (type: int)
             outputColumnNames: key, c_int, p, q, x, b
             Statistics: Num rows: 1458 Data size: 389286 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator


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

Posted by ga...@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/hbase-metastore
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


[49/50] [abbrv] hive git commit: HIVE-11343 Merge branch 'master' into hbase-metastore

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/61db7b80/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/61db7b80/itests/hive-unit/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/61db7b80/metastore/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/61db7b80/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/61db7b80/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/61db7b80/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/61db7b80/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------
diff --cc metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index e5a32fb,39ab9e7..d2177a5
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@@ -299,8 -338,27 +339,28 @@@ public class ObjectStore implements Raw
          " created in the thread with id: " + Thread.currentThread().getId());
    }
  
+   /**
+    * Creates the proxy used to evaluate expressions. This is here to prevent circular
+    * dependency - ql -&gt; metastore client &lt;-&gt metastore server -&gt ql. If server and
+    * client are split, this can be removed.
+    * @param conf Configuration.
+    * @return The partition expression proxy.
+    */
+   private static PartitionExpressionProxy createExpressionProxy(Configuration conf) {
+     String className = HiveConf.getVar(conf, HiveConf.ConfVars.METASTORE_EXPRESSION_PROXY_CLASS);
+     try {
+       @SuppressWarnings("unchecked")
+       Class<? extends PartitionExpressionProxy> clazz =
+           (Class<? extends PartitionExpressionProxy>)MetaStoreUtils.getClass(className);
+       return MetaStoreUtils.newInstance(
+           clazz, new Class<?>[0], new Object[0]);
+     } catch (MetaException e) {
+       LOG.error("Error loading PartitionExpressionProxy", e);
+       throw new RuntimeException("Error loading PartitionExpressionProxy: " + e.getMessage());
+     }
+   }
  
 +
    /**
     * Properties specified in hive-default.xml override the properties specified
     * in jpox.properties.
@@@ -3258,8 -3426,9 +3350,9 @@@
          if (roleMember.size() > 0) {
            pm.deletePersistentAll(roleMember);
          }
+         queryWrapper.close();
          // then remove all the grants
 -        List<MGlobalPrivilege> userGrants = listPrincipalGlobalGrants(
 +        List<MGlobalPrivilege> userGrants = listPrincipalMGlobalGrants(
              mRol.getRoleName(), PrincipalType.ROLE);
          if (userGrants.size() > 0) {
            pm.deletePersistentAll(userGrants);
@@@ -3344,26 -3521,27 +3445,27 @@@
    }
  
    @SuppressWarnings("unchecked")
 -  @Override
 -  public List<MRoleMap> listRoles(String principalName, PrincipalType principalType) {
 +  public List<MRoleMap> listMRoles(String principalName,
 +      PrincipalType principalType) {
      boolean success = false;
-     List<MRoleMap> mRoleMember = null;
+     Query query = null;
+     List<MRoleMap> mRoleMember = new ArrayList<MRoleMap>();
+ 
      try {
-       openTransaction();
        LOG.debug("Executing listRoles");
-       Query query = pm
-           .newQuery(
-               MRoleMap.class,
-               "principalName == t1 && principalType == t2");
-       query
-           .declareParameters("java.lang.String t1, java.lang.String t2");
+ 
+       openTransaction();
+       query = pm.newQuery(MRoleMap.class, "principalName == t1 && principalType == t2");
+       query.declareParameters("java.lang.String t1, java.lang.String t2");
        query.setUnique(false);
-       mRoleMember = (List<MRoleMap>) query.executeWithArray(
-           principalName, principalType.toString());
-       LOG.debug("Done executing query for listMSecurityUserRoleMap");
-       pm.retrieveAll(mRoleMember);
+       List<MRoleMap> mRoles =
+           (List<MRoleMap>) query.executeWithArray(principalName, principalType.toString());
+       pm.retrieveAll(mRoles);
        success = commitTransaction();
-       LOG.debug("Done retrieving all objects for listMSecurityUserRoleMap");
+ 
+       mRoleMember.addAll(mRoles);
+ 
+       LOG.debug("Done retrieving all objects for listRoles");
      } finally {
        if (!success) {
          rollbackTransaction();
@@@ -3372,60 -3553,18 +3477,56 @@@
  
      if (principalType == PrincipalType.USER) {
        // All users belong to public role implicitly, add that role
-       if (mRoleMember == null) {
-         mRoleMember = new ArrayList<MRoleMap>();
-       } else {
-         mRoleMember = new ArrayList<MRoleMap>(mRoleMember);
-       }
        MRole publicRole = new MRole(HiveMetaStore.PUBLIC, 0, HiveMetaStore.PUBLIC);
-       mRoleMember.add(new MRoleMap(principalName, principalType.toString(), publicRole, 0,
-           null, null, false));
+       mRoleMember.add(new MRoleMap(principalName, principalType.toString(), publicRole, 0, null,
+           null, false));
      }
-     return mRoleMember;
  
+     return mRoleMember;
    }
  
 +  @Override
 +  public List<Role> listRoles(String principalName, PrincipalType principalType) {
 +    List<Role> result = new ArrayList<Role>();
 +    List<MRoleMap> roleMaps = listMRoles(principalName, principalType);
 +    if (roleMaps != null) {
 +      for (MRoleMap roleMap : roleMaps) {
 +        MRole mrole = roleMap.getRole();
 +        Role role = new Role(mrole.getRoleName(), mrole.getCreateTime(), mrole.getOwnerName());
 +        result.add(role);
 +      }
 +    }
 +    return result;
 +  }
 +
 +  @Override
 +  public List<RolePrincipalGrant> listRolesWithGrants(String principalName,
 +                                                      PrincipalType principalType) {
 +    List<RolePrincipalGrant> result = new ArrayList<RolePrincipalGrant>();
 +    List<MRoleMap> roleMaps = listMRoles(principalName, principalType);
 +    if (roleMaps != null) {
 +      for (MRoleMap roleMap : roleMaps) {
 +        RolePrincipalGrant rolePrinGrant = new RolePrincipalGrant(
 +            roleMap.getRole().getRoleName(),
 +            roleMap.getPrincipalName(),
 +            PrincipalType.valueOf(roleMap.getPrincipalType()),
 +            roleMap.getGrantOption(),
 +            roleMap.getAddTime(),
 +            roleMap.getGrantor(),
 +            // no grantor type for public role, hence the null check
 +            roleMap.getGrantorType() == null ? null
 +                : PrincipalType.valueOf(roleMap.getGrantorType())
 +        );
 +        result.add(rolePrinGrant);
 +      }
 +    }
 +    return result;
 +  }
 +
    @SuppressWarnings("unchecked")
    private List<MRoleMap> listMSecurityPrincipalMembershipRole(final String roleName,
-       final PrincipalType principalType) {
+       final PrincipalType principalType,
+       QueryWrapper queryWrapper) {
      boolean success = false;
      List<MRoleMap> mRoleMemebership = null;
      try {
@@@ -4305,22 -4448,25 +4411,24 @@@
    }
  
    @SuppressWarnings("unchecked")
 -  @Override
 -  public List<MRoleMap> listRoleMembers(String roleName) {
 +  public List<MRoleMap> listMRoleMembers(String roleName) {
      boolean success = false;
-     List<MRoleMap> mRoleMemeberList = null;
+     Query query = null;
+     List<MRoleMap> mRoleMemeberList = new ArrayList<MRoleMap>();
      try {
+       LOG.debug("Executing listRoleMembers");
+ 
        openTransaction();
-       LOG.debug("Executing listMSecurityUserRoleMember");
-       Query query = pm.newQuery(MRoleMap.class,
-           "role.roleName == t1");
+       query = pm.newQuery(MRoleMap.class, "role.roleName == t1");
        query.declareParameters("java.lang.String t1");
        query.setUnique(false);
-       mRoleMemeberList = (List<MRoleMap>) query.execute(
-           roleName);
-       LOG.debug("Done executing query for listMSecurityUserRoleMember");
-       pm.retrieveAll(mRoleMemeberList);
+       List<MRoleMap> mRoles = (List<MRoleMap>) query.execute(roleName);
+       pm.retrieveAll(mRoles);
        success = commitTransaction();
-       LOG.debug("Done retrieving all objects for listMSecurityUserRoleMember");
+ 
+       mRoleMemeberList.addAll(mRoles);
+ 
+       LOG.debug("Done retrieving all objects for listRoleMembers");
      } finally {
        if (!success) {
          rollbackTransaction();
@@@ -4329,48 -4478,27 +4440,51 @@@
      return mRoleMemeberList;
    }
  
 -  @SuppressWarnings("unchecked")
    @Override
 -  public List<MGlobalPrivilege> listPrincipalGlobalGrants(String principalName,
 -      PrincipalType principalType) {
 +  public List<RolePrincipalGrant> listRoleMembers(String roleName) {
 +    List<MRoleMap> roleMaps = listMRoleMembers(roleName);
 +    List<RolePrincipalGrant> rolePrinGrantList = new ArrayList<RolePrincipalGrant>();
 +
 +    if (roleMaps != null) {
 +      for (MRoleMap roleMap : roleMaps) {
 +        RolePrincipalGrant rolePrinGrant = new RolePrincipalGrant(
 +            roleMap.getRole().getRoleName(),
 +            roleMap.getPrincipalName(),
 +            PrincipalType.valueOf(roleMap.getPrincipalType()),
 +            roleMap.getGrantOption(),
 +            roleMap.getAddTime(),
 +            roleMap.getGrantor(),
 +            // no grantor type for public role, hence the null check
 +            roleMap.getGrantorType() == null ? null
 +                : PrincipalType.valueOf(roleMap.getGrantorType())
 +        );
 +        rolePrinGrantList.add(rolePrinGrant);
 +
 +      }
 +    }
 +    return rolePrinGrantList;
 +  }
 +
 +  @SuppressWarnings("unchecked")
 +  public List<MGlobalPrivilege> listPrincipalMGlobalGrants(String principalName,
 +                                                           PrincipalType principalType) {
      boolean commited = false;
-     List<MGlobalPrivilege> userNameDbPriv = null;
+     Query query = null;
+     List<MGlobalPrivilege> userNameDbPriv = new ArrayList<MGlobalPrivilege>();
      try {
+       List<MGlobalPrivilege> mPrivs = null;
        openTransaction();
        if (principalName != null) {
-         Query query = pm.newQuery(MGlobalPrivilege.class,
-             "principalName == t1 && principalType == t2 ");
-         query.declareParameters(
-             "java.lang.String t1, java.lang.String t2");
-         userNameDbPriv = (List<MGlobalPrivilege>) query
-             .executeWithArray(principalName, principalType.toString());
-         pm.retrieveAll(userNameDbPriv);
+         query = pm.newQuery(MGlobalPrivilege.class, "principalName == t1 && principalType == t2 ");
+         query.declareParameters("java.lang.String t1, java.lang.String t2");
+         mPrivs = (List<MGlobalPrivilege>) query
+                 .executeWithArray(principalName, principalType.toString());
+         pm.retrieveAll(mPrivs);
        }
        commited = commitTransaction();
+       if (mPrivs != null) {
+         userNameDbPriv.addAll(mPrivs);
+       }
      } finally {
        if (!commited) {
          rollbackTransaction();
@@@ -4380,34 -4511,12 +4497,35 @@@
    }
  
    @Override
 +  public List<HiveObjectPrivilege> listPrincipalGlobalGrants(String principalName,
 +                                                             PrincipalType principalType) {
 +    List<MGlobalPrivilege> mUsers =
 +        listPrincipalMGlobalGrants(principalName, principalType);
 +    if (mUsers.isEmpty()) {
 +      return Collections.<HiveObjectPrivilege> emptyList();
 +    }
 +    List<HiveObjectPrivilege> result = new ArrayList<HiveObjectPrivilege>();
 +    for (int i = 0; i < mUsers.size(); i++) {
 +      MGlobalPrivilege sUsr = mUsers.get(i);
 +      HiveObjectRef objectRef = new HiveObjectRef(
 +          HiveObjectType.GLOBAL, null, null, null, null);
 +      HiveObjectPrivilege secUser = new HiveObjectPrivilege(
 +          objectRef, sUsr.getPrincipalName(), principalType,
 +          new PrivilegeGrantInfo(sUsr.getPrivilege(), sUsr
 +              .getCreateTime(), sUsr.getGrantor(), PrincipalType
 +              .valueOf(sUsr.getGrantorType()), sUsr.getGrantOption()));
 +      result.add(secUser);
 +    }
 +    return result;
 +  }
 +
 +  @Override
    public List<HiveObjectPrivilege> listGlobalGrantsAll() {
      boolean commited = false;
+     Query query = null;
      try {
        openTransaction();
-       Query query = pm.newQuery(MGlobalPrivilege.class);
+       query = pm.newQuery(MGlobalPrivilege.class);
        List<MGlobalPrivilege> userNameDbPriv = (List<MGlobalPrivilege>) query.execute();
        pm.retrieveAll(userNameDbPriv);
        commited = commitTransaction();
@@@ -4435,23 -4547,28 +4556,27 @@@
    }
  
    @SuppressWarnings("unchecked")
 -  @Override
 -  public List<MDBPrivilege> listPrincipalDBGrants(String principalName,
 +  public List<MDBPrivilege> listPrincipalMDBGrants(String principalName,
        PrincipalType principalType, String dbName) {
      boolean success = false;
-     List<MDBPrivilege> mSecurityDBList = null;
+     Query query = null;
+     List<MDBPrivilege> mSecurityDBList = new ArrayList<MDBPrivilege>();
      dbName = HiveStringUtils.normalizeIdentifier(dbName);
- 
      try {
-       openTransaction();
        LOG.debug("Executing listPrincipalDBGrants");
-         Query query = pm.newQuery(MDBPrivilege.class,
-             "principalName == t1 && principalType == t2 && database.name == t3");
-         query
-             .declareParameters("java.lang.String t1, java.lang.String t2, java.lang.String t3");
-         mSecurityDBList = (List<MDBPrivilege>) query.executeWithArray(principalName, principalType.toString(), dbName);
-       LOG.debug("Done executing query for listPrincipalDBGrants");
-       pm.retrieveAll(mSecurityDBList);
+ 
+       openTransaction();
+       query =
+           pm.newQuery(MDBPrivilege.class,
+               "principalName == t1 && principalType == t2 && database.name == t3");
+       query.declareParameters("java.lang.String t1, java.lang.String t2, java.lang.String t3");
+       List<MDBPrivilege> mPrivs =
+           (List<MDBPrivilege>) query.executeWithArray(principalName, principalType.toString(),
+               dbName);
+       pm.retrieveAll(mPrivs);
        success = commitTransaction();
+ 
+       mSecurityDBList.addAll(mPrivs);
        LOG.debug("Done retrieving all objects for listPrincipalDBGrants");
      } finally {
        if (!success) {
@@@ -4462,32 -4582,14 +4590,37 @@@
    }
  
    @Override
 +  public List<HiveObjectPrivilege> listPrincipalDBGrants(String principalName,
 +                                                         PrincipalType principalType,
 +                                                         String dbName) {
 +    List<MDBPrivilege> mDbs = listPrincipalMDBGrants(principalName, principalType, dbName);
 +    if (mDbs.isEmpty()) {
 +      return Collections.<HiveObjectPrivilege>emptyList();
 +    }
 +    List<HiveObjectPrivilege> result = new ArrayList<HiveObjectPrivilege>();
 +    for (int i = 0; i < mDbs.size(); i++) {
 +      MDBPrivilege sDB = mDbs.get(i);
 +      HiveObjectRef objectRef = new HiveObjectRef(
 +          HiveObjectType.DATABASE, dbName, null, null, null);
 +      HiveObjectPrivilege secObj = new HiveObjectPrivilege(objectRef,
 +          sDB.getPrincipalName(), principalType,
 +          new PrivilegeGrantInfo(sDB.getPrivilege(), sDB
 +              .getCreateTime(), sDB.getGrantor(), PrincipalType
 +              .valueOf(sDB.getGrantorType()), sDB.getGrantOption()));
 +      result.add(secObj);
 +    }
 +    return result;
 +  }
 +
 +  @Override
    public List<HiveObjectPrivilege> listPrincipalDBGrantsAll(
        String principalName, PrincipalType principalType) {
-     return convertDB(listPrincipalAllDBGrant(principalName, principalType));
+     QueryWrapper queryWrapper = new QueryWrapper();
+     try {
+       return convertDB(listPrincipalAllDBGrant(principalName, principalType, queryWrapper));
+     } finally {
+       queryWrapper.close();
+     }
    }
  
    @Override
@@@ -4781,15 -4904,15 +4935,15 @@@
      return new ObjectPair<Query, Object[]>(query, params);
    }
  
 -  @Override
    @SuppressWarnings("unchecked")
 -  public List<MTablePrivilege> listAllTableGrants(String principalName,
 -      PrincipalType principalType, String dbName, String tableName) {
 +  public List<MTablePrivilege> listAllMTableGrants(
 +      String principalName, PrincipalType principalType, String dbName,
 +      String tableName) {
      tableName = HiveStringUtils.normalizeIdentifier(tableName);
      dbName = HiveStringUtils.normalizeIdentifier(dbName);
- 
      boolean success = false;
-     List<MTablePrivilege> mSecurityTabPartList = null;
+     Query query = null;
+     List<MTablePrivilege> mSecurityTabPartList = new ArrayList<MTablePrivilege>();
      try {
        openTransaction();
        LOG.debug("Executing listAllTableGrants");
@@@ -4813,57 -4941,35 +4972,60 @@@
      return mSecurityTabPartList;
    }
  
 -  @SuppressWarnings("unchecked")
    @Override
 -  public List<MPartitionPrivilege> listPrincipalPartitionGrants(String principalName,
 -      PrincipalType principalType, String dbName, String tableName, String partName) {
 +  public List<HiveObjectPrivilege> listAllTableGrants(String principalName,
 +                                                      PrincipalType principalType,
 +                                                      String dbName,
 +                                                      String tableName) {
 +    List<MTablePrivilege> mTbls =
 +        listAllMTableGrants(principalName, principalType, dbName, tableName);
 +    if (mTbls.isEmpty()) {
 +      return Collections.<HiveObjectPrivilege> emptyList();
 +    }
 +    List<HiveObjectPrivilege> result = new ArrayList<HiveObjectPrivilege>();
 +    for (int i = 0; i < mTbls.size(); i++) {
 +      MTablePrivilege sTbl = mTbls.get(i);
 +      HiveObjectRef objectRef = new HiveObjectRef(
 +          HiveObjectType.TABLE, dbName, tableName, null, null);
 +      HiveObjectPrivilege secObj = new HiveObjectPrivilege(objectRef,
 +          sTbl.getPrincipalName(), principalType,
 +          new PrivilegeGrantInfo(sTbl.getPrivilege(), sTbl.getCreateTime(), sTbl
 +              .getGrantor(), PrincipalType.valueOf(sTbl
 +              .getGrantorType()), sTbl.getGrantOption()));
 +      result.add(secObj);
 +    }
 +    return result;
 +  }
 +
 +  @SuppressWarnings("unchecked")
 +  public List<MPartitionPrivilege> listPrincipalMPartitionGrants(
 +      String principalName, PrincipalType principalType, String dbName,
 +      String tableName, String partName) {
      boolean success = false;
+     Query query = null;
      tableName = HiveStringUtils.normalizeIdentifier(tableName);
      dbName = HiveStringUtils.normalizeIdentifier(dbName);
- 
-     List<MPartitionPrivilege> mSecurityTabPartList = null;
+     List<MPartitionPrivilege> mSecurityTabPartList = new ArrayList<MPartitionPrivilege>();
      try {
-       openTransaction();
-       LOG.debug("Executing listMSecurityPrincipalPartitionGrant");
-       Query query = pm.newQuery(
-           MPartitionPrivilege.class,
-               "principalName == t1 && principalType == t2 && partition.table.tableName == t3 " +
-               "&& partition.table.database.name == t4 && partition.partitionName == t5");
-       query.declareParameters(
-           "java.lang.String t1, java.lang.String t2, java.lang.String t3, java.lang.String t4, " +
-           "java.lang.String t5");
-       mSecurityTabPartList = (List<MPartitionPrivilege>) query
-           .executeWithArray(principalName, principalType.toString(), tableName, dbName, partName);
-       LOG.debug("Done executing query for listMSecurityPrincipalPartitionGrant");
+       LOG.debug("Executing listPrincipalPartitionGrants");
  
-       pm.retrieveAll(mSecurityTabPartList);
+       openTransaction();
+       query =
+           pm.newQuery(MPartitionPrivilege.class,
+               "principalName == t1 && principalType == t2 && partition.table.tableName == t3 "
+                   + "&& partition.table.database.name == t4 && partition.partitionName == t5");
+       query
+           .declareParameters("java.lang.String t1, java.lang.String t2, java.lang.String t3, java.lang.String t4, "
+               + "java.lang.String t5");
+       List<MPartitionPrivilege> mPrivs =
+           (List<MPartitionPrivilege>) query.executeWithArray(principalName,
+               principalType.toString(), tableName, dbName, partName);
+       pm.retrieveAll(mPrivs);
        success = commitTransaction();
-       LOG.debug("Done retrieving all objects for listMSecurityPrincipalPartitionGrant");
+ 
+       mSecurityTabPartList.addAll(mPrivs);
+ 
+       LOG.debug("Done retrieving all objects for listPrincipalPartitionGrants");
      } finally {
        if (!success) {
          rollbackTransaction();
@@@ -4872,40 -4981,12 +5037,41 @@@
      return mSecurityTabPartList;
    }
  
 -  @SuppressWarnings("unchecked")
    @Override
 -  public List<MTableColumnPrivilege> listPrincipalTableColumnGrants(String principalName,
 -      PrincipalType principalType, String dbName, String tableName, String columnName) {
 +  public List<HiveObjectPrivilege> listPrincipalPartitionGrants(String principalName,
 +                                                                PrincipalType principalType,
 +                                                                String dbName,
 +                                                                String tableName,
 +                                                                List<String> partValues,
 +                                                                String partName) {
 +    List<MPartitionPrivilege> mParts = listPrincipalMPartitionGrants(principalName,
 +        principalType, dbName, tableName, partName);
 +    if (mParts.isEmpty()) {
 +      return Collections.<HiveObjectPrivilege> emptyList();
 +    }
 +    List<HiveObjectPrivilege> result = new ArrayList<HiveObjectPrivilege>();
 +    for (int i = 0; i < mParts.size(); i++) {
 +      MPartitionPrivilege sPart = mParts.get(i);
 +      HiveObjectRef objectRef = new HiveObjectRef(
 +          HiveObjectType.PARTITION, dbName, tableName, partValues, null);
 +      HiveObjectPrivilege secObj = new HiveObjectPrivilege(objectRef,
 +          sPart.getPrincipalName(), principalType,
 +          new PrivilegeGrantInfo(sPart.getPrivilege(), sPart
 +              .getCreateTime(), sPart.getGrantor(), PrincipalType
 +              .valueOf(sPart.getGrantorType()), sPart
 +              .getGrantOption()));
 +
 +      result.add(secObj);
 +    }
 +    return result;
 +  }
 +
 +  @SuppressWarnings("unchecked")
 +  public List<MTableColumnPrivilege> listPrincipalMTableColumnGrants(
 +      String principalName, PrincipalType principalType, String dbName,
 +      String tableName, String columnName) {
      boolean success = false;
+     Query query = null;
      tableName = HiveStringUtils.normalizeIdentifier(tableName);
      dbName = HiveStringUtils.normalizeIdentifier(dbName);
      columnName = HiveStringUtils.normalizeIdentifier(columnName);
@@@ -4935,37 -5022,12 +5107,38 @@@
    }
  
    @Override
 +  public List<HiveObjectPrivilege> listPrincipalTableColumnGrants(String principalName,
 +                                                                  PrincipalType principalType,
 +                                                                  String dbName,
 +                                                                  String tableName,
 +                                                                  String columnName) {
 +    List<MTableColumnPrivilege> mTableCols =
 +        listPrincipalMTableColumnGrants(principalName, principalType, dbName, tableName, columnName);
 +    if (mTableCols.isEmpty()) {
 +      return Collections.emptyList();
 +    }
 +    List<HiveObjectPrivilege> result = new ArrayList<HiveObjectPrivilege>();
 +    for (int i = 0; i < mTableCols.size(); i++) {
 +      MTableColumnPrivilege sCol = mTableCols.get(i);
 +      HiveObjectRef objectRef = new HiveObjectRef(
 +          HiveObjectType.COLUMN, dbName, tableName, null, sCol.getColumnName());
 +      HiveObjectPrivilege secObj = new HiveObjectPrivilege(
 +          objectRef, sCol.getPrincipalName(), principalType,
 +          new PrivilegeGrantInfo(sCol.getPrivilege(), sCol
 +              .getCreateTime(), sCol.getGrantor(), PrincipalType
 +              .valueOf(sCol.getGrantorType()), sCol
 +              .getGrantOption()));
 +      result.add(secObj);
 +    }
 +    return result;
 +  }
 +
    @SuppressWarnings("unchecked")
 -  public List<MPartitionColumnPrivilege> listPrincipalPartitionColumnGrants(String principalName,
 -      PrincipalType principalType, String dbName, String tableName, String partitionName,
 -      String columnName) {
 +  public List<MPartitionColumnPrivilege> listPrincipalMPartitionColumnGrants(
 +      String principalName, PrincipalType principalType, String dbName,
 +      String tableName, String partitionName, String columnName) {
      boolean success = false;
+     Query query = null;
      tableName = HiveStringUtils.normalizeIdentifier(tableName);
      dbName = HiveStringUtils.normalizeIdentifier(dbName);
      columnName = HiveStringUtils.normalizeIdentifier(columnName);
@@@ -5001,38 -5063,10 +5174,39 @@@
    }
  
    @Override
 -  public List<HiveObjectPrivilege> listPrincipalPartitionColumnGrantsAll(String principalName,
 -      PrincipalType principalType) {
 +  public List<HiveObjectPrivilege> listPrincipalPartitionColumnGrants(String principalName,
 +                                                                      PrincipalType principalType,
 +                                                                      String dbName,
 +                                                                      String tableName,
 +                                                                      List<String> partValues,
 +                                                                      String partitionName,
 +                                                                      String columnName) {
 +    List<MPartitionColumnPrivilege> mPartitionCols =
 +        listPrincipalMPartitionColumnGrants(principalName, principalType, dbName, tableName,
 +            partitionName, columnName);
 +    if (mPartitionCols.isEmpty()) {
 +      return Collections.emptyList();
 +    }
 +    List<HiveObjectPrivilege> result = new ArrayList<HiveObjectPrivilege>();
 +    for (int i = 0; i < mPartitionCols.size(); i++) {
 +      MPartitionColumnPrivilege sCol = mPartitionCols.get(i);
 +      HiveObjectRef objectRef = new HiveObjectRef(
 +          HiveObjectType.COLUMN, dbName, tableName, partValues, sCol.getColumnName());
 +      HiveObjectPrivilege secObj = new HiveObjectPrivilege(objectRef,
 +          sCol.getPrincipalName(), principalType,
 +          new PrivilegeGrantInfo(sCol.getPrivilege(), sCol
 +              .getCreateTime(), sCol.getGrantor(), PrincipalType
 +              .valueOf(sCol.getGrantorType()), sCol.getGrantOption()));
 +      result.add(secObj);
 +    }
 +    return result;
 +  }
 +
 +  @Override
 +  public List<HiveObjectPrivilege> listPrincipalPartitionColumnGrantsAll(
 +      String principalName, PrincipalType principalType) {
      boolean success = false;
+     Query query = null;
      try {
        openTransaction();
        LOG.debug("Executing listPrincipalPartitionColumnGrantsAll");
@@@ -6294,16 -6400,11 +6540,16 @@@
      }.run(true);
    }
  
 -  private List<MPartitionColumnStatistics> getMPartitionColumnStatistics(Table table,
 -      List<String> partNames, List<String> colNames,
 -      QueryWrapper queryWrapper) throws NoSuchObjectException, MetaException {
 +  @Override
 +  public void flushCache() {
 +    // NOP as there's no caching
 +  }
 +
 +  private List<MPartitionColumnStatistics> getMPartitionColumnStatistics(
-       Table table, List<String> partNames, List<String> colNames)
++      Table table, List<String> partNames, List<String> colNames, QueryWrapper queryWrapper)
 +          throws NoSuchObjectException, MetaException {
      boolean committed = false;
-     MPartitionColumnStatistics mStatsObj = null;
+ 
      try {
        openTransaction();
        // We are not going to verify SD for each partition. Just verify for the table.

http://git-wip-us.apache.org/repos/asf/hive/blob/61db7b80/pom.xml
----------------------------------------------------------------------
diff --cc pom.xml
index 7d763fe,1abf738..e9c054a
--- a/pom.xml
+++ b/pom.xml
@@@ -172,7 -174,7 +174,8 @@@
      <felix.version>2.4.0</felix.version>
      <curator.version>2.6.0</curator.version>
      <jsr305.version>3.0.0</jsr305.version>
 +    <tephra.version>0.4.0</tephra.version>
+     <gson.version>2.2.4</gson.version>
    </properties>
  
    <repositories>

http://git-wip-us.apache.org/repos/asf/hive/blob/61db7b80/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/61db7b80/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/61db7b80/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/61db7b80/ql/src/java/org/apache/hadoop/hive/ql/metadata/Partition.java
----------------------------------------------------------------------


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

Posted by ga...@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/hbase-metastore
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();
+  }
+  
+}


[13/50] [abbrv] hive git commit: HIVE-11145 Remove OFFLINE and NO_DROP from tables and partitions (gates, reviewed by Ashutosh Chauhan)

Posted by ga...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/stats_partscan_1_23.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/stats_partscan_1_23.q.out b/ql/src/test/results/clientpositive/stats_partscan_1_23.q.out
index 5488935..2a292fe 100644
--- a/ql/src/test/results/clientpositive/stats_partscan_1_23.q.out
+++ b/ql/src/test/results/clientpositive/stats_partscan_1_23.q.out
@@ -75,8 +75,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart_partial_scan	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   
@@ -150,8 +148,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart_partial_scan	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -192,8 +188,6 @@ Partition Value:    	[2008-04-09, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart_partial_scan	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/statsfs.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/statsfs.q.out b/ql/src/test/results/clientpositive/statsfs.q.out
index b0bca41..2735f5f 100644
--- a/ql/src/test/results/clientpositive/statsfs.q.out
+++ b/ql/src/test/results/clientpositive/statsfs.q.out
@@ -65,8 +65,6 @@ Partition Value:    	[2010]
 Database:           	default             	 
 Table:              	t1                  	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -106,8 +104,6 @@ Partition Value:    	[2011]
 Database:           	default             	 
 Table:              	t1                  	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -187,8 +183,6 @@ Partition Value:    	[2010]
 Database:           	default             	 
 Table:              	t1                  	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -228,8 +222,6 @@ Partition Value:    	[2011]
 Database:           	default             	 
 Table:              	t1                  	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -298,7 +290,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -364,7 +355,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -457,8 +447,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	t1                  	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -499,8 +487,6 @@ Partition Value:    	[2008-04-09, 12]
 Database:           	default             	 
 Table:              	t1                  	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/tez/alter_merge_stats_orc.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/alter_merge_stats_orc.q.out b/ql/src/test/results/clientpositive/tez/alter_merge_stats_orc.q.out
index 43cc4ef..cefe069 100644
--- a/ql/src/test/results/clientpositive/tez/alter_merge_stats_orc.q.out
+++ b/ql/src/test/results/clientpositive/tez/alter_merge_stats_orc.q.out
@@ -85,7 +85,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -137,7 +136,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -242,8 +240,6 @@ Partition Value:    	[2011]
 Database:           	default             	 
 Table:              	src_orc_merge_test_part_stat	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	3                   
@@ -293,8 +289,6 @@ Partition Value:    	[2011]
 Database:           	default             	 
 Table:              	src_orc_merge_test_part_stat	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	3                   
@@ -352,8 +346,6 @@ Partition Value:    	[2011]
 Database:           	default             	 
 Table:              	src_orc_merge_test_part_stat	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/tez/ctas.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/ctas.q.out b/ql/src/test/results/clientpositive/tez/ctas.q.out
index 5e9aee2..27b189d 100644
--- a/ql/src/test/results/clientpositive/tez/ctas.q.out
+++ b/ql/src/test/results/clientpositive/tez/ctas.q.out
@@ -146,7 +146,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -294,7 +293,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -442,7 +440,6 @@ conb                	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -507,7 +504,6 @@ conb                	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -656,7 +652,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/tez/dynpart_sort_opt_vectorization.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/dynpart_sort_opt_vectorization.q.out b/ql/src/test/results/clientpositive/tez/dynpart_sort_opt_vectorization.q.out
index 22afcbf..4451046 100644
--- a/ql/src/test/results/clientpositive/tez/dynpart_sort_opt_vectorization.q.out
+++ b/ql/src/test/results/clientpositive/tez/dynpart_sort_opt_vectorization.q.out
@@ -941,8 +941,6 @@ Partition Value:    	[foo, 27]
 Database:           	default             	 
 Table:              	over1k_part_orc     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   
@@ -985,8 +983,6 @@ Partition Value:    	[foo, __HIVE_DEFAULT_PARTITION__]
 Database:           	default             	 
 Table:              	over1k_part_orc     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   
@@ -1029,8 +1025,6 @@ Partition Value:    	[foo, 27]
 Database:           	default             	 
 Table:              	over1k_part_limit_orc	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   
@@ -1073,8 +1067,6 @@ Partition Value:    	[foo, __HIVE_DEFAULT_PARTITION__]
 Database:           	default             	 
 Table:              	over1k_part_limit_orc	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   
@@ -1116,8 +1108,6 @@ Partition Value:    	[27]
 Database:           	default             	 
 Table:              	over1k_part_buck_orc	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	8                   
@@ -1159,8 +1149,6 @@ Partition Value:    	[__HIVE_DEFAULT_PARTITION__]
 Database:           	default             	 
 Table:              	over1k_part_buck_orc	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	8                   
@@ -1202,8 +1190,6 @@ Partition Value:    	[27]
 Database:           	default             	 
 Table:              	over1k_part_buck_sort_orc	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	8                   
@@ -1245,8 +1231,6 @@ Partition Value:    	[__HIVE_DEFAULT_PARTITION__]
 Database:           	default             	 
 Table:              	over1k_part_buck_sort_orc	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	8                   
@@ -1791,8 +1775,6 @@ Partition Value:    	[foo, 27]
 Database:           	default             	 
 Table:              	over1k_part2_orc    	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1835,8 +1817,6 @@ Partition Value:    	[foo, __HIVE_DEFAULT_PARTITION__]
 Database:           	default             	 
 Table:              	over1k_part2_orc    	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1940,8 +1920,6 @@ Partition Value:    	[foo, 27]
 Database:           	default             	 
 Table:              	over1k_part2_orc    	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1984,8 +1962,6 @@ Partition Value:    	[foo, __HIVE_DEFAULT_PARTITION__]
 Database:           	default             	 
 Table:              	over1k_part2_orc    	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -2256,8 +2232,6 @@ Partition Value:    	[27]
 Database:           	default             	 
 Table:              	over1k_part_buck_sort2_orc	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -2299,8 +2273,6 @@ Partition Value:    	[__HIVE_DEFAULT_PARTITION__]
 Database:           	default             	 
 Table:              	over1k_part_buck_sort2_orc	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -2475,8 +2447,6 @@ Partition Value:    	[27]
 Database:           	default             	 
 Table:              	over1k_part_buck_sort2_orc	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -2518,8 +2488,6 @@ Partition Value:    	[__HIVE_DEFAULT_PARTITION__]
 Database:           	default             	 
 Table:              	over1k_part_buck_sort2_orc	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/tez/dynpart_sort_optimization.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/dynpart_sort_optimization.q.out b/ql/src/test/results/clientpositive/tez/dynpart_sort_optimization.q.out
index 8a16645..cb001b9 100644
--- a/ql/src/test/results/clientpositive/tez/dynpart_sort_optimization.q.out
+++ b/ql/src/test/results/clientpositive/tez/dynpart_sort_optimization.q.out
@@ -854,8 +854,6 @@ Partition Value:    	[foo, 27]
 Database:           	default             	 
 Table:              	over1k_part         	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   
@@ -898,8 +896,6 @@ Partition Value:    	[foo, __HIVE_DEFAULT_PARTITION__]
 Database:           	default             	 
 Table:              	over1k_part         	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   
@@ -942,8 +938,6 @@ Partition Value:    	[foo, 27]
 Database:           	default             	 
 Table:              	over1k_part_limit   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   
@@ -986,8 +980,6 @@ Partition Value:    	[foo, __HIVE_DEFAULT_PARTITION__]
 Database:           	default             	 
 Table:              	over1k_part_limit   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	2                   
@@ -1029,8 +1021,6 @@ Partition Value:    	[27]
 Database:           	default             	 
 Table:              	over1k_part_buck    	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	8                   
@@ -1072,8 +1062,6 @@ Partition Value:    	[__HIVE_DEFAULT_PARTITION__]
 Database:           	default             	 
 Table:              	over1k_part_buck    	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	8                   
@@ -1115,8 +1103,6 @@ Partition Value:    	[27]
 Database:           	default             	 
 Table:              	over1k_part_buck_sort	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	8                   
@@ -1158,8 +1144,6 @@ Partition Value:    	[__HIVE_DEFAULT_PARTITION__]
 Database:           	default             	 
 Table:              	over1k_part_buck_sort	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	8                   
@@ -1691,8 +1675,6 @@ Partition Value:    	[foo, 27]
 Database:           	default             	 
 Table:              	over1k_part2        	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1735,8 +1717,6 @@ Partition Value:    	[foo, __HIVE_DEFAULT_PARTITION__]
 Database:           	default             	 
 Table:              	over1k_part2        	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1840,8 +1820,6 @@ Partition Value:    	[foo, 27]
 Database:           	default             	 
 Table:              	over1k_part2        	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1884,8 +1862,6 @@ Partition Value:    	[foo, __HIVE_DEFAULT_PARTITION__]
 Database:           	default             	 
 Table:              	over1k_part2        	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -2152,8 +2128,6 @@ Partition Value:    	[27]
 Database:           	default             	 
 Table:              	over1k_part_buck_sort2	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -2195,8 +2169,6 @@ Partition Value:    	[__HIVE_DEFAULT_PARTITION__]
 Database:           	default             	 
 Table:              	over1k_part_buck_sort2	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -2299,8 +2271,6 @@ Partition Value:    	[27]
 Database:           	default             	 
 Table:              	over1k_part_buck_sort2	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -2342,8 +2312,6 @@ Partition Value:    	[__HIVE_DEFAULT_PARTITION__]
 Database:           	default             	 
 Table:              	over1k_part_buck_sort2	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/tez/dynpart_sort_optimization2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/dynpart_sort_optimization2.q.out b/ql/src/test/results/clientpositive/tez/dynpart_sort_optimization2.q.out
index 1f741aa..5bba0cb 100644
--- a/ql/src/test/results/clientpositive/tez/dynpart_sort_optimization2.q.out
+++ b/ql/src/test/results/clientpositive/tez/dynpart_sort_optimization2.q.out
@@ -197,8 +197,6 @@ Partition Value:    	[2452617]
 Database:           	default             	 
 Table:              	ss_part             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -259,8 +257,6 @@ Partition Value:    	[2452638]
 Database:           	default             	 
 Table:              	ss_part             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -428,8 +424,6 @@ Partition Value:    	[2452617]
 Database:           	default             	 
 Table:              	ss_part             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -490,8 +484,6 @@ Partition Value:    	[2452638]
 Database:           	default             	 
 Table:              	ss_part             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -680,8 +672,6 @@ Partition Value:    	[2452617]
 Database:           	default             	 
 Table:              	ss_part             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -742,8 +732,6 @@ Partition Value:    	[2452638]
 Database:           	default             	 
 Table:              	ss_part             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -910,8 +898,6 @@ Partition Value:    	[2452617]
 Database:           	default             	 
 Table:              	ss_part             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -972,8 +958,6 @@ Partition Value:    	[2452638]
 Database:           	default             	 
 Table:              	ss_part             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1219,8 +1203,6 @@ Partition Value:    	[2452617]
 Database:           	default             	 
 Table:              	ss_part_orc         	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1281,8 +1263,6 @@ Partition Value:    	[2452638]
 Database:           	default             	 
 Table:              	ss_part_orc         	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1451,8 +1431,6 @@ Partition Value:    	[2452617]
 Database:           	default             	 
 Table:              	ss_part_orc         	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1513,8 +1491,6 @@ Partition Value:    	[2452638]
 Database:           	default             	 
 Table:              	ss_part_orc         	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/tez/orc_analyze.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/orc_analyze.q.out b/ql/src/test/results/clientpositive/tez/orc_analyze.q.out
index a61a2e6..6eb9a93 100644
--- a/ql/src/test/results/clientpositive/tez/orc_analyze.q.out
+++ b/ql/src/test/results/clientpositive/tez/orc_analyze.q.out
@@ -98,7 +98,6 @@ state               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -147,7 +146,6 @@ state               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -196,7 +194,6 @@ state               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -286,7 +283,6 @@ state               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -404,8 +400,6 @@ Partition Value:    	[Ca]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -449,8 +443,6 @@ Partition Value:    	[Or]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -506,8 +498,6 @@ Partition Value:    	[Ca]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -551,8 +541,6 @@ Partition Value:    	[Or]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -608,8 +596,6 @@ Partition Value:    	[Ca]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -653,8 +639,6 @@ Partition Value:    	[Or]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -755,8 +739,6 @@ Partition Value:    	[Ca]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -800,8 +782,6 @@ Partition Value:    	[Or]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -922,8 +902,6 @@ Partition Value:    	[Ca]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -967,8 +945,6 @@ Partition Value:    	[Or]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1024,8 +1000,6 @@ Partition Value:    	[Ca]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1069,8 +1043,6 @@ Partition Value:    	[Or]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1126,8 +1098,6 @@ Partition Value:    	[Ca]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1171,8 +1141,6 @@ Partition Value:    	[Or]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1279,8 +1247,6 @@ Partition Value:    	[Ca]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1324,8 +1290,6 @@ Partition Value:    	[Or]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1491,8 +1455,6 @@ Partition Value:    	[Ca]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1536,8 +1498,6 @@ Partition Value:    	[OH]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   
@@ -1595,8 +1555,6 @@ Partition Value:    	[Ca]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1640,8 +1598,6 @@ Partition Value:    	[OH]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   
@@ -1699,8 +1655,6 @@ Partition Value:    	[Ca]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -1744,8 +1698,6 @@ Partition Value:    	[OH]
 Database:           	default             	 
 Table:              	orc_create_people   	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/tez/selectDistinctStar.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/selectDistinctStar.q.out b/ql/src/test/results/clientpositive/tez/selectDistinctStar.q.out
index 59c8f2a..590db13 100644
--- a/ql/src/test/results/clientpositive/tez/selectDistinctStar.q.out
+++ b/ql/src/test/results/clientpositive/tez/selectDistinctStar.q.out
@@ -1383,7 +1383,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 Table Type:         	VIRTUAL_VIEW        	 
 Table Parameters:	 	 
@@ -3834,7 +3833,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 Table Type:         	VIRTUAL_VIEW        	 
 Table Parameters:	 	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/tez/stats_counter.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/stats_counter.q.out b/ql/src/test/results/clientpositive/tez/stats_counter.q.out
index e2980e8..8b3dcea 100644
--- a/ql/src/test/results/clientpositive/tez/stats_counter.q.out
+++ b/ql/src/test/results/clientpositive/tez/stats_counter.q.out
@@ -32,7 +32,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -80,7 +79,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/tez/stats_counter_partitioned.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/stats_counter_partitioned.q.out b/ql/src/test/results/clientpositive/tez/stats_counter_partitioned.q.out
index ab1270c..626dcff 100644
--- a/ql/src/test/results/clientpositive/tez/stats_counter_partitioned.q.out
+++ b/ql/src/test/results/clientpositive/tez/stats_counter_partitioned.q.out
@@ -66,8 +66,6 @@ Partition Value:    	[2008, 11]
 Database:           	default             	 
 Table:              	dummy               	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -108,8 +106,6 @@ Partition Value:    	[2008, 12]
 Database:           	default             	 
 Table:              	dummy               	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -190,8 +186,6 @@ Partition Value:    	[10, 11]
 Database:           	default             	 
 Table:              	dummy               	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -232,8 +226,6 @@ Partition Value:    	[10, 12]
 Database:           	default             	 
 Table:              	dummy               	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -323,8 +315,6 @@ Partition Value:    	[1997]
 Database:           	default             	 
 Table:              	dummy               	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -363,8 +353,6 @@ Partition Value:    	[1994]
 Database:           	default             	 
 Table:              	dummy               	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -403,8 +391,6 @@ Partition Value:    	[1998]
 Database:           	default             	 
 Table:              	dummy               	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -443,8 +429,6 @@ Partition Value:    	[1996]
 Database:           	default             	 
 Table:              	dummy               	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/tez/stats_noscan_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/stats_noscan_1.q.out b/ql/src/test/results/clientpositive/tez/stats_noscan_1.q.out
index aa49526..96e0e43 100644
--- a/ql/src/test/results/clientpositive/tez/stats_noscan_1.q.out
+++ b/ql/src/test/results/clientpositive/tez/stats_noscan_1.q.out
@@ -100,8 +100,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -142,8 +140,6 @@ Partition Value:    	[2008-04-08, 12]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -184,8 +180,6 @@ Partition Value:    	[2008-04-09, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   
@@ -226,8 +220,6 @@ Partition Value:    	[2008-04-09, 12]
 Database:           	default             	 
 Table:              	analyze_srcpart     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   
@@ -266,7 +258,6 @@ hr                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -381,8 +372,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart_partial	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -423,8 +412,6 @@ Partition Value:    	[2008-04-08, 12]
 Database:           	default             	 
 Table:              	analyze_srcpart_partial	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -465,8 +452,6 @@ Partition Value:    	[2008-04-09, 11]
 Database:           	default             	 
 Table:              	analyze_srcpart_partial	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   
@@ -507,8 +492,6 @@ Partition Value:    	[2008-04-09, 12]
 Database:           	default             	 
 Table:              	analyze_srcpart_partial	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/tez/stats_only_null.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/stats_only_null.q.out b/ql/src/test/results/clientpositive/tez/stats_only_null.q.out
index 2725be0..eabda89 100644
--- a/ql/src/test/results/clientpositive/tez/stats_only_null.q.out
+++ b/ql/src/test/results/clientpositive/tez/stats_only_null.q.out
@@ -229,8 +229,6 @@ Partition Value:    	[2010]
 Database:           	default             	 
 Table:              	stats_null_part     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -272,8 +270,6 @@ Partition Value:    	[2011]
 Database:           	default             	 
 Table:              	stats_null_part     	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/tez/tez_fsstat.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/tez_fsstat.q.out b/ql/src/test/results/clientpositive/tez/tez_fsstat.q.out
index 3fcf68c..50666a9 100644
--- a/ql/src/test/results/clientpositive/tez/tez_fsstat.q.out
+++ b/ql/src/test/results/clientpositive/tez/tez_fsstat.q.out
@@ -82,8 +82,6 @@ Partition Value:    	[2008-04-08]
 Database:           	default             	 
 Table:              	tab_part            	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	4                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/truncate_column.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/truncate_column.q.out b/ql/src/test/results/clientpositive/truncate_column.q.out
index 2d3e378..2b99d86 100644
--- a/ql/src/test/results/clientpositive/truncate_column.q.out
+++ b/ql/src/test/results/clientpositive/truncate_column.q.out
@@ -36,7 +36,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -100,7 +99,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -176,7 +174,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -242,7 +239,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -318,7 +314,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -383,7 +378,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -450,7 +444,6 @@ value               	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -534,8 +527,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_tab_part       	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
@@ -603,8 +594,6 @@ Partition Value:    	[1]
 Database:           	default             	 
 Table:              	test_tab_part       	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/unicode_notation.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/unicode_notation.q.out b/ql/src/test/results/clientpositive/unicode_notation.q.out
index 5849705..52da674 100644
--- a/ql/src/test/results/clientpositive/unicode_notation.q.out
+++ b/ql/src/test/results/clientpositive/unicode_notation.q.out
@@ -23,7 +23,6 @@ a                   	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -70,7 +69,6 @@ a                   	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
@@ -117,7 +115,6 @@ a                   	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/union_remove_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union_remove_1.q.out b/ql/src/test/results/clientpositive/union_remove_1.q.out
index 18307fc..35e4458 100644
--- a/ql/src/test/results/clientpositive/union_remove_1.q.out
+++ b/ql/src/test/results/clientpositive/union_remove_1.q.out
@@ -188,7 +188,6 @@ values              	bigint
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/union_remove_10.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union_remove_10.q.out b/ql/src/test/results/clientpositive/union_remove_10.q.out
index abc6b8b..2159b7e 100644
--- a/ql/src/test/results/clientpositive/union_remove_10.q.out
+++ b/ql/src/test/results/clientpositive/union_remove_10.q.out
@@ -247,7 +247,6 @@ values              	bigint
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/union_remove_11.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union_remove_11.q.out b/ql/src/test/results/clientpositive/union_remove_11.q.out
index 362e093..2ab83dc 100644
--- a/ql/src/test/results/clientpositive/union_remove_11.q.out
+++ b/ql/src/test/results/clientpositive/union_remove_11.q.out
@@ -236,7 +236,6 @@ values              	bigint
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/union_remove_12.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union_remove_12.q.out b/ql/src/test/results/clientpositive/union_remove_12.q.out
index 74cb2a4..82dbcdb 100644
--- a/ql/src/test/results/clientpositive/union_remove_12.q.out
+++ b/ql/src/test/results/clientpositive/union_remove_12.q.out
@@ -227,7 +227,6 @@ values              	bigint
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/union_remove_13.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union_remove_13.q.out b/ql/src/test/results/clientpositive/union_remove_13.q.out
index b12d7cd..5d8433e 100644
--- a/ql/src/test/results/clientpositive/union_remove_13.q.out
+++ b/ql/src/test/results/clientpositive/union_remove_13.q.out
@@ -250,7 +250,6 @@ values              	bigint
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/union_remove_14.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union_remove_14.q.out b/ql/src/test/results/clientpositive/union_remove_14.q.out
index 02abe09..4760f29 100644
--- a/ql/src/test/results/clientpositive/union_remove_14.q.out
+++ b/ql/src/test/results/clientpositive/union_remove_14.q.out
@@ -229,7 +229,6 @@ values              	bigint
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/union_remove_15.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union_remove_15.q.out b/ql/src/test/results/clientpositive/union_remove_15.q.out
index 76f8c6c..a259df8 100644
--- a/ql/src/test/results/clientpositive/union_remove_15.q.out
+++ b/ql/src/test/results/clientpositive/union_remove_15.q.out
@@ -212,7 +212,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/union_remove_16.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union_remove_16.q.out b/ql/src/test/results/clientpositive/union_remove_16.q.out
index 28c4c2d..c7a08f3 100644
--- a/ql/src/test/results/clientpositive/union_remove_16.q.out
+++ b/ql/src/test/results/clientpositive/union_remove_16.q.out
@@ -244,7 +244,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/union_remove_17.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union_remove_17.q.out b/ql/src/test/results/clientpositive/union_remove_17.q.out
index 476ca3a..688e365 100644
--- a/ql/src/test/results/clientpositive/union_remove_17.q.out
+++ b/ql/src/test/results/clientpositive/union_remove_17.q.out
@@ -167,7 +167,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/union_remove_18.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union_remove_18.q.out b/ql/src/test/results/clientpositive/union_remove_18.q.out
index 27986ba..96daa12 100644
--- a/ql/src/test/results/clientpositive/union_remove_18.q.out
+++ b/ql/src/test/results/clientpositive/union_remove_18.q.out
@@ -222,7 +222,6 @@ ds                  	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/union_remove_19.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union_remove_19.q.out b/ql/src/test/results/clientpositive/union_remove_19.q.out
index 2a1fd55..c1f688e 100644
--- a/ql/src/test/results/clientpositive/union_remove_19.q.out
+++ b/ql/src/test/results/clientpositive/union_remove_19.q.out
@@ -192,7 +192,6 @@ values              	bigint
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/union_remove_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union_remove_2.q.out b/ql/src/test/results/clientpositive/union_remove_2.q.out
index 78a6a48..29e5d6c 100644
--- a/ql/src/test/results/clientpositive/union_remove_2.q.out
+++ b/ql/src/test/results/clientpositive/union_remove_2.q.out
@@ -199,7 +199,6 @@ values              	bigint
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/union_remove_20.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union_remove_20.q.out b/ql/src/test/results/clientpositive/union_remove_20.q.out
index 922ba4c..1da81a7 100644
--- a/ql/src/test/results/clientpositive/union_remove_20.q.out
+++ b/ql/src/test/results/clientpositive/union_remove_20.q.out
@@ -198,7 +198,6 @@ key                 	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/union_remove_21.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union_remove_21.q.out b/ql/src/test/results/clientpositive/union_remove_21.q.out
index 4eac30d..4743d8d 100644
--- a/ql/src/test/results/clientpositive/union_remove_21.q.out
+++ b/ql/src/test/results/clientpositive/union_remove_21.q.out
@@ -182,7 +182,6 @@ key                 	string
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/union_remove_22.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union_remove_22.q.out b/ql/src/test/results/clientpositive/union_remove_22.q.out
index 01cce7a..3f13991 100644
--- a/ql/src/test/results/clientpositive/union_remove_22.q.out
+++ b/ql/src/test/results/clientpositive/union_remove_22.q.out
@@ -202,7 +202,6 @@ values2             	bigint
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/union_remove_23.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union_remove_23.q.out b/ql/src/test/results/clientpositive/union_remove_23.q.out
index 71ffa1b..cdbe914 100644
--- a/ql/src/test/results/clientpositive/union_remove_23.q.out
+++ b/ql/src/test/results/clientpositive/union_remove_23.q.out
@@ -230,7 +230,6 @@ values              	bigint
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/union_remove_24.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union_remove_24.q.out b/ql/src/test/results/clientpositive/union_remove_24.q.out
index 7eaff6f..49086e4 100644
--- a/ql/src/test/results/clientpositive/union_remove_24.q.out
+++ b/ql/src/test/results/clientpositive/union_remove_24.q.out
@@ -194,7 +194,6 @@ values              	bigint
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/union_remove_25.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union_remove_25.q.out b/ql/src/test/results/clientpositive/union_remove_25.q.out
index 78c1f07..37d6a53 100644
--- a/ql/src/test/results/clientpositive/union_remove_25.q.out
+++ b/ql/src/test/results/clientpositive/union_remove_25.q.out
@@ -213,8 +213,6 @@ Partition Value:    	[2004]
 Database:           	default             	 
 Table:              	outputtbl1          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	2                   
@@ -419,8 +417,6 @@ Partition Value:    	[2008-04-08]
 Database:           	default             	 
 Table:              	outputtbl2          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	2                   
@@ -609,8 +605,6 @@ Partition Value:    	[2008-04-08, 11]
 Database:           	default             	 
 Table:              	outputtbl3          	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
-#### A masked pattern was here ####
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	false               
 	numFiles            	2                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/union_remove_3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union_remove_3.q.out b/ql/src/test/results/clientpositive/union_remove_3.q.out
index f4cbf7b..7045a26 100644
--- a/ql/src/test/results/clientpositive/union_remove_3.q.out
+++ b/ql/src/test/results/clientpositive/union_remove_3.q.out
@@ -188,7 +188,6 @@ values              	bigint
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/union_remove_4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union_remove_4.q.out b/ql/src/test/results/clientpositive/union_remove_4.q.out
index 1946cb6..c545dd4 100644
--- a/ql/src/test/results/clientpositive/union_remove_4.q.out
+++ b/ql/src/test/results/clientpositive/union_remove_4.q.out
@@ -232,7 +232,6 @@ values              	bigint
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/union_remove_5.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union_remove_5.q.out b/ql/src/test/results/clientpositive/union_remove_5.q.out
index b065ba0..1308c09 100644
--- a/ql/src/test/results/clientpositive/union_remove_5.q.out
+++ b/ql/src/test/results/clientpositive/union_remove_5.q.out
@@ -245,7 +245,6 @@ values              	bigint
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/union_remove_7.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union_remove_7.q.out b/ql/src/test/results/clientpositive/union_remove_7.q.out
index e00ba3a..61bef8b 100644
--- a/ql/src/test/results/clientpositive/union_remove_7.q.out
+++ b/ql/src/test/results/clientpositive/union_remove_7.q.out
@@ -192,7 +192,6 @@ values              	bigint
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/union_remove_8.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union_remove_8.q.out b/ql/src/test/results/clientpositive/union_remove_8.q.out
index 64cf98b..62af170 100644
--- a/ql/src/test/results/clientpositive/union_remove_8.q.out
+++ b/ql/src/test/results/clientpositive/union_remove_8.q.out
@@ -203,7 +203,6 @@ values              	bigint
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d6ec52ee/ql/src/test/results/clientpositive/union_remove_9.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union_remove_9.q.out b/ql/src/test/results/clientpositive/union_remove_9.q.out
index bf7f8de..c0fc54d 100644
--- a/ql/src/test/results/clientpositive/union_remove_9.q.out
+++ b/ql/src/test/results/clientpositive/union_remove_9.q.out
@@ -250,7 +250,6 @@ values              	bigint
 # Detailed Table Information	 	 
 Database:           	default             	 
 #### A masked pattern was here ####
-Protect Mode:       	None                	 
 Retention:          	0                   	 
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 


[47/50] [abbrv] 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 ga...@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/hbase-metastore
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;
 


[28/50] [abbrv] hive git commit: HIVE-11279 : Hive should emit lineage information in json compact format (Lenni Kuff via Szehon)

Posted by ga...@apache.org.
HIVE-11279 : Hive should emit lineage information in json compact format (Lenni Kuff via Szehon)


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

Branch: refs/heads/hbase-metastore
Commit: 2b1f03e33d2bbfc6916e80caff963475c2740f82
Parents: 7788968
Author: Szehon Ho <sz...@cloudera.com>
Authored: Fri Jul 17 11:19:25 2015 -0700
Committer: Szehon Ho <sz...@cloudera.com>
Committed: Fri Jul 17 11:19:55 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hive/ql/hooks/LineageLogger.java     |    4 +-
 .../test/results/clientpositive/lineage2.q.out  | 2296 +-----------------
 .../test/results/clientpositive/lineage3.q.out  | 2235 +----------------
 3 files changed, 58 insertions(+), 4477 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/2b1f03e3/ql/src/java/org/apache/hadoop/hive/ql/hooks/LineageLogger.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/LineageLogger.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/LineageLogger.java
index fc32af7..d615372 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/hooks/LineageLogger.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/LineageLogger.java
@@ -137,9 +137,7 @@ public class LineageLogger implements ExecuteWithHookContext {
       try {
         StringBuilderWriter out = new StringBuilderWriter(1024);
         JsonWriter writer = new JsonWriter(out);
-        writer.setIndent("  ");
 
-        out.append("POSTHOOK: LINEAGE: ");
         String queryStr = plan.getQueryStr().trim();
         writer.beginObject();
         writer.name("version").value(FORMAT_VERSION);
@@ -182,7 +180,7 @@ public class LineageLogger implements ExecuteWithHookContext {
           // Log to console
           log(lineage);
         } else {
-          // In none test mode, emit to a log file,
+          // In non-test mode, emit to a log file,
           // which can be different from the normal hive.log.
           // For example, using NoDeleteRollingFileAppender to
           // log to some file with different rolling policy.

http://git-wip-us.apache.org/repos/asf/hive/blob/2b1f03e3/ql/src/test/results/clientpositive/lineage2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/lineage2.q.out b/ql/src/test/results/clientpositive/lineage2.q.out
index 669be97..9b227c6 100644
--- a/ql/src/test/results/clientpositive/lineage2.q.out
+++ b/ql/src/test/results/clientpositive/lineage2.q.out
@@ -5,118 +5,12 @@ PREHOOK: type: CREATETABLE_AS_SELECT
 PREHOOK: Input: default@src1
 PREHOOK: Output: database:default
 PREHOOK: Output: default@src2
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "3a39d46286e4c2cd2139c9bb248f7b4f",
-  "queryText": "create table src2 as select key key2, value value2 from src1",
-  "edges": [
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        3
-      ],
-      "targets": [
-        1
-      ],
-      "edgeType": "PROJECTION"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src2.key2"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src2.value2"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.key"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.value"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"3a39d46286e4c2cd2139c9bb248f7b4f","queryText":"create table src2 as select key key2, value value2 from src1","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.src2.key2"},{"id":1,"vertexType":"COLUMN","vertexId":"default.src2.value2"},{"id":2,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"default.src1.value"}]}
 PREHOOK: query: select * from src1 where key is not null and value is not null limit 3
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src1
 #### A masked pattern was here ####
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "b5b224847b2333e790a2c229434a04c8",
-  "queryText": "select * from src1 where key is not null and value is not null limit 3",
-  "edges": [
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        3
-      ],
-      "targets": [
-        1
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        2,
-        3
-      ],
-      "targets": [
-        0,
-        1
-      ],
-      "expression": "(src1.key is not null and src1.value is not null)",
-      "edgeType": "PREDICATE"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "src1.key"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "src1.value"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.key"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.value"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"b5b224847b2333e790a2c229434a04c8","queryText":"select * from src1 where key is not null and value is not null limit 3","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"},{"sources":[2,3],"targets":[0,1],"expression":"(src1.key is not null and src1.value is not null)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"src1.key"},{"id":1,"vertexType":"COLUMN","vertexId":"src1.value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"default.src1.value"}]}
 238	val_238
 	
 311	val_311
@@ -124,66 +18,7 @@ PREHOOK: query: select * from src1 where key > 10 and value > 'val' order by key
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src1
 #### A masked pattern was here ####
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "773d9d0ea92e797eae292ae1eeea11ab",
-  "queryText": "select * from src1 where key > 10 and value > 'val' order by key limit 5",
-  "edges": [
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        3
-      ],
-      "targets": [
-        1
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        2,
-        3
-      ],
-      "targets": [
-        0,
-        1
-      ],
-      "expression": "((UDFToDouble(src1.key) > UDFToDouble(10)) and (src1.value > 'val'))",
-      "edgeType": "PREDICATE"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "src1.key"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "src1.value"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.key"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.value"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"773d9d0ea92e797eae292ae1eeea11ab","queryText":"select * from src1 where key > 10 and value > 'val' order by key limit 5","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"},{"sources":[2,3],"targets":[0,1],"expression":"((UDFToDouble(src1.key) > UDFToDouble(10)) and (src1.value > 'val'))","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"src1.key"},{"id":1,"vertexType":"COLUMN","vertexId":"src1.value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"default.src1.value"}]}
 146	val_146
 150	val_150
 213	val_213
@@ -196,158 +31,17 @@ PREHOOK: type: CREATETABLE_AS_SELECT
 PREHOOK: Input: default@src1
 PREHOOK: Output: database:default
 PREHOOK: Output: default@dest1
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "712fe958c357bcfc978b95c43eb19084",
-  "queryText": "create table dest1 as select * from src1",
-  "edges": [
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        3
-      ],
-      "targets": [
-        1
-      ],
-      "edgeType": "PROJECTION"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest1.key"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest1.value"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.key"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.value"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"712fe958c357bcfc978b95c43eb19084","queryText":"create table dest1 as select * from src1","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.dest1.key"},{"id":1,"vertexType":"COLUMN","vertexId":"default.dest1.value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"default.src1.value"}]}
 PREHOOK: query: insert into table dest1 select * from src2
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src2
 PREHOOK: Output: default@dest1
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "ecc718a966d8887b18084a55dd96f0bc",
-  "queryText": "insert into table dest1 select * from src2",
-  "edges": [
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        3
-      ],
-      "targets": [
-        1
-      ],
-      "edgeType": "PROJECTION"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest1.key"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest1.value"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src2.key2"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src2.value2"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"ecc718a966d8887b18084a55dd96f0bc","queryText":"insert into table dest1 select * from src2","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.dest1.key"},{"id":1,"vertexType":"COLUMN","vertexId":"default.dest1.value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.src2.key2"},{"id":3,"vertexType":"COLUMN","vertexId":"default.src2.value2"}]}
 PREHOOK: query: select key k, dest1.value from dest1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@dest1
 #### A masked pattern was here ####
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "416b6f4cd63edd4f9d8213d2d7819d21",
-  "queryText": "select key k, dest1.value from dest1",
-  "edges": [
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        3
-      ],
-      "targets": [
-        1
-      ],
-      "edgeType": "PROJECTION"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "k"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "dest1.value"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest1.key"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest1.value"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"416b6f4cd63edd4f9d8213d2d7819d21","queryText":"select key k, dest1.value from dest1","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"k"},{"id":1,"vertexType":"COLUMN","vertexId":"dest1.value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.dest1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"default.dest1.value"}]}
 238	val_238
 	
 311	val_311
@@ -403,42 +97,7 @@ PREHOOK: type: QUERY
 PREHOOK: Input: default@src1
 PREHOOK: Input: default@src2
 #### A masked pattern was here ####
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "50fa3d1074b3fda37ce11dc6ec92ebf3",
-  "queryText": "select key from src1 union select key2 from src2 order by key",
-  "edges": [
-    {
-      "sources": [
-        1,
-        2
-      ],
-      "targets": [
-        0
-      ],
-      "expression": "key",
-      "edgeType": "PROJECTION"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "u2.key"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.key"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src2.key2"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"50fa3d1074b3fda37ce11dc6ec92ebf3","queryText":"select key from src1 union select key2 from src2 order by key","edges":[{"sources":[1,2],"targets":[0],"expression":"key","edgeType":"PROJECTION"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"u2.key"},{"id":1,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":2,"vertexType":"COLUMN","vertexId":"default.src2.key2"}]}
 
 128
 146
@@ -460,42 +119,7 @@ PREHOOK: type: QUERY
 PREHOOK: Input: default@src1
 PREHOOK: Input: default@src2
 #### A masked pattern was here ####
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "a739460bd79c8c91ec35e22c97329769",
-  "queryText": "select key k from src1 union select key2 from src2 order by k",
-  "edges": [
-    {
-      "sources": [
-        1,
-        2
-      ],
-      "targets": [
-        0
-      ],
-      "expression": "key",
-      "edgeType": "PROJECTION"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "u2.k"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.key"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src2.key2"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"a739460bd79c8c91ec35e22c97329769","queryText":"select key k from src1 union select key2 from src2 order by k","edges":[{"sources":[1,2],"targets":[0],"expression":"key","edgeType":"PROJECTION"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"u2.k"},{"id":1,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":2,"vertexType":"COLUMN","vertexId":"default.src2.key2"}]}
 
 128
 146
@@ -516,55 +140,7 @@ PREHOOK: query: select key, count(1) a from dest1 group by key
 PREHOOK: type: QUERY
 PREHOOK: Input: default@dest1
 #### A masked pattern was here ####
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "3901b5e3a164064736b3234355046340",
-  "queryText": "select key, count(1) a from dest1 group by key",
-  "edges": [
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        3
-      ],
-      "targets": [
-        1
-      ],
-      "expression": "count(1)",
-      "edgeType": "PROJECTION"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "key"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "a"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest1.key"
-    },
-    {
-      "id": 3,
-      "vertexType": "TABLE",
-      "vertexId": "default.dest1"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"3901b5e3a164064736b3234355046340","queryText":"select key, count(1) a from dest1 group by key","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"expression":"count(1)","edgeType":"PROJECTION"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"key"},{"id":1,"vertexType":"COLUMN","vertexId":"a"},{"id":2,"vertexType":"COLUMN","vertexId":"default.dest1.key"},{"id":3,"vertexType":"TABLE","vertexId":"default.dest1"}]}
 	20
 128	2
 146	2
@@ -585,55 +161,7 @@ PREHOOK: query: select key k, count(*) from dest1 group by key
 PREHOOK: type: QUERY
 PREHOOK: Input: default@dest1
 #### A masked pattern was here ####
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "0d5a212f10847aeaab31e8c31121e6d4",
-  "queryText": "select key k, count(*) from dest1 group by key",
-  "edges": [
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        3
-      ],
-      "targets": [
-        1
-      ],
-      "expression": "count(*)",
-      "edgeType": "PROJECTION"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "k"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "c1"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest1.key"
-    },
-    {
-      "id": 3,
-      "vertexType": "TABLE",
-      "vertexId": "default.dest1"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"0d5a212f10847aeaab31e8c31121e6d4","queryText":"select key k, count(*) from dest1 group by key","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"expression":"count(*)","edgeType":"PROJECTION"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"k"},{"id":1,"vertexType":"COLUMN","vertexId":"c1"},{"id":2,"vertexType":"COLUMN","vertexId":"default.dest1.key"},{"id":3,"vertexType":"TABLE","vertexId":"default.dest1"}]}
 	20
 128	2
 146	2
@@ -654,55 +182,7 @@ PREHOOK: query: select key k, count(value) from dest1 group by key
 PREHOOK: type: QUERY
 PREHOOK: Input: default@dest1
 #### A masked pattern was here ####
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "56429eccb04ded722f5bd9d9d8cf7260",
-  "queryText": "select key k, count(value) from dest1 group by key",
-  "edges": [
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        3
-      ],
-      "targets": [
-        1
-      ],
-      "expression": "count(default.dest1.value)",
-      "edgeType": "PROJECTION"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "k"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "c1"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest1.key"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest1.value"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"56429eccb04ded722f5bd9d9d8cf7260","queryText":"select key k, count(value) from dest1 group by key","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"expression":"count(default.dest1.value)","edgeType":"PROJECTION"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"k"},{"id":1,"vertexType":"COLUMN","vertexId":"c1"},{"id":2,"vertexType":"COLUMN","vertexId":"default.dest1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"default.dest1.value"}]}
 	20
 128	2
 146	2
@@ -723,55 +203,7 @@ PREHOOK: query: select value, max(length(key)) from dest1 group by value
 PREHOOK: type: QUERY
 PREHOOK: Input: default@dest1
 #### A masked pattern was here ####
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "7e1cfc3dece85b41b6f7c46365580cde",
-  "queryText": "select value, max(length(key)) from dest1 group by value",
-  "edges": [
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        3
-      ],
-      "targets": [
-        1
-      ],
-      "expression": "max(length(dest1.key))",
-      "edgeType": "PROJECTION"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "value"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "c1"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest1.value"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest1.key"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"7e1cfc3dece85b41b6f7c46365580cde","queryText":"select value, max(length(key)) from dest1 group by value","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"expression":"max(length(dest1.key))","edgeType":"PROJECTION"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"value"},{"id":1,"vertexType":"COLUMN","vertexId":"c1"},{"id":2,"vertexType":"COLUMN","vertexId":"default.dest1.value"},{"id":3,"vertexType":"COLUMN","vertexId":"default.dest1.key"}]}
 	3
 val_146	3
 val_150	3
@@ -795,55 +227,7 @@ PREHOOK: query: select value, max(length(key)) from dest1 group by value order b
 PREHOOK: type: QUERY
 PREHOOK: Input: default@dest1
 #### A masked pattern was here ####
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "c6578ce1dd72498c4af33f20f164e483",
-  "queryText": "select value, max(length(key)) from dest1 group by value order by value limit 5",
-  "edges": [
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        3
-      ],
-      "targets": [
-        1
-      ],
-      "expression": "max(length(dest1.key))",
-      "edgeType": "PROJECTION"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "value"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "c1"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest1.value"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest1.key"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"c6578ce1dd72498c4af33f20f164e483","queryText":"select value, max(length(key)) from dest1 group by value order by value limit 5","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"expression":"max(length(dest1.key))","edgeType":"PROJECTION"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"value"},{"id":1,"vertexType":"COLUMN","vertexId":"c1"},{"id":2,"vertexType":"COLUMN","vertexId":"default.dest1.value"},{"id":3,"vertexType":"COLUMN","vertexId":"default.dest1.key"}]}
 	3
 val_146	3
 val_150	3
@@ -853,55 +237,7 @@ PREHOOK: query: select key, length(value) from dest1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@dest1
 #### A masked pattern was here ####
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "91fbcea5cb34362071555cd93e8d0abe",
-  "queryText": "select key, length(value) from dest1",
-  "edges": [
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        3
-      ],
-      "targets": [
-        1
-      ],
-      "expression": "length(dest1.value)",
-      "edgeType": "PROJECTION"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "key"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "c1"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest1.key"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest1.value"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"91fbcea5cb34362071555cd93e8d0abe","queryText":"select key, length(value) from dest1","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"expression":"length(dest1.value)","edgeType":"PROJECTION"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"key"},{"id":1,"vertexType":"COLUMN","vertexId":"c1"},{"id":2,"vertexType":"COLUMN","vertexId":"default.dest1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"default.dest1.value"}]}
 238	7
 	0
 311	7
@@ -956,36 +292,7 @@ PREHOOK: query: select length(value) + 3 from dest1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@dest1
 #### A masked pattern was here ####
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "3d8a347cc9052111cb328938d37b9b03",
-  "queryText": "select length(value) + 3 from dest1",
-  "edges": [
-    {
-      "sources": [
-        1
-      ],
-      "targets": [
-        0
-      ],
-      "expression": "(length(dest1.value) + 3)",
-      "edgeType": "PROJECTION"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "c0"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest1.value"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"3d8a347cc9052111cb328938d37b9b03","queryText":"select length(value) + 3 from dest1","edges":[{"sources":[1],"targets":[0],"expression":"(length(dest1.value) + 3)","edgeType":"PROJECTION"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"c0"},{"id":1,"vertexType":"COLUMN","vertexId":"default.dest1.value"}]}
 10
 3
 10
@@ -1040,29 +347,7 @@ PREHOOK: query: select 5 from dest1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@dest1
 #### A masked pattern was here ####
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "bae960bf4376ec00e37258469b17360d",
-  "queryText": "select 5 from dest1",
-  "edges": [
-    {
-      "sources": [],
-      "targets": [
-        0
-      ],
-      "expression": "5",
-      "edgeType": "PROJECTION"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "c0"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"bae960bf4376ec00e37258469b17360d","queryText":"select 5 from dest1","edges":[{"sources":[],"targets":[0],"expression":"5","edgeType":"PROJECTION"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"c0"}]}
 5
 5
 5
@@ -1117,29 +402,7 @@ PREHOOK: query: select 3 * 5 from dest1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@dest1
 #### A masked pattern was here ####
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "753abad4d55afd3df34fdc73abfcd44d",
-  "queryText": "select 3 * 5 from dest1",
-  "edges": [
-    {
-      "sources": [],
-      "targets": [
-        0
-      ],
-      "expression": "(3 * 5)",
-      "edgeType": "PROJECTION"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "c0"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"753abad4d55afd3df34fdc73abfcd44d","queryText":"select 3 * 5 from dest1","edges":[{"sources":[],"targets":[0],"expression":"(3 * 5)","edgeType":"PROJECTION"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"c0"}]}
 15
 15
 15
@@ -1198,485 +461,31 @@ PREHOOK: Input: default@src1
 PREHOOK: Input: default@src2
 PREHOOK: Output: database:default
 PREHOOK: Output: default@dest2
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "386791c174a4999fc916e300b5e76bf2",
-  "queryText": "create table dest2 as select * from src1 JOIN src2 ON src1.key = src2.key2",
-  "edges": [
-    {
-      "sources": [
-        4
-      ],
-      "targets": [
-        0
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        5
-      ],
-      "targets": [
-        1
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        6
-      ],
-      "targets": [
-        2
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        7
-      ],
-      "targets": [
-        3
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        4,
-        6
-      ],
-      "targets": [
-        0,
-        1,
-        2,
-        3
-      ],
-      "expression": "(src1.key = src2.key2)",
-      "edgeType": "PREDICATE"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest2.key"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest2.value"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest2.key2"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest2.value2"
-    },
-    {
-      "id": 4,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.key"
-    },
-    {
-      "id": 5,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.value"
-    },
-    {
-      "id": 6,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src2.key2"
-    },
-    {
-      "id": 7,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src2.value2"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"386791c174a4999fc916e300b5e76bf2","queryText":"create table dest2 as select * from src1 JOIN src2 ON src1.key = src2.key2","edges":[{"sources":[4],"targets":[0],"edgeType":"PROJECTION"},{"sources":[5],"targets":[1],"edgeType":"PROJECTION"},{"sources":[6],"targets":[2],"edgeType":"PROJECTION"},{"sources":[7],"targets":[3],"edgeType":"PROJECTION"},{"sources":[4,6],"targets":[0,1,2,3],"expression":"(src1.key = src2.key2)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.dest2.key"},{"id":1,"vertexType":"COLUMN","vertexId":"default.dest2.value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.dest2.key2"},{"id":3,"vertexType":"COLUMN","vertexId":"default.dest2.value2"},{"id":4,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":5,"vertexType":"COLUMN","vertexId":"default.src1.value"},{"id":6,"vertexType":"COLUMN","vertexId":"default.src2.key2"},{"id":7,"vertexType":"COLUMN","vertexId":"default.src2.val
 ue2"}]}
 PREHOOK: query: insert overwrite table dest2 select * from src1 JOIN src2 ON src1.key = src2.key2
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src1
 PREHOOK: Input: default@src2
 PREHOOK: Output: default@dest2
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "e494b771d94800dc3430bf5d0810cd9f",
-  "queryText": "insert overwrite table dest2 select * from src1 JOIN src2 ON src1.key = src2.key2",
-  "edges": [
-    {
-      "sources": [
-        4
-      ],
-      "targets": [
-        0
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        5
-      ],
-      "targets": [
-        1
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        6
-      ],
-      "targets": [
-        2
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        7
-      ],
-      "targets": [
-        3
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        4,
-        6
-      ],
-      "targets": [
-        0,
-        1,
-        2,
-        3
-      ],
-      "expression": "(src1.key = src2.key2)",
-      "edgeType": "PREDICATE"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest2.key"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest2.value"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest2.key2"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest2.value2"
-    },
-    {
-      "id": 4,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.key"
-    },
-    {
-      "id": 5,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.value"
-    },
-    {
-      "id": 6,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src2.key2"
-    },
-    {
-      "id": 7,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src2.value2"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"e494b771d94800dc3430bf5d0810cd9f","queryText":"insert overwrite table dest2 select * from src1 JOIN src2 ON src1.key = src2.key2","edges":[{"sources":[4],"targets":[0],"edgeType":"PROJECTION"},{"sources":[5],"targets":[1],"edgeType":"PROJECTION"},{"sources":[6],"targets":[2],"edgeType":"PROJECTION"},{"sources":[7],"targets":[3],"edgeType":"PROJECTION"},{"sources":[4,6],"targets":[0,1,2,3],"expression":"(src1.key = src2.key2)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.dest2.key"},{"id":1,"vertexType":"COLUMN","vertexId":"default.dest2.value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.dest2.key2"},{"id":3,"vertexType":"COLUMN","vertexId":"default.dest2.value2"},{"id":4,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":5,"vertexType":"COLUMN","vertexId":"default.src1.value"},{"id":6,"vertexType":"COLUMN","vertexId":"default.src2.key2"},{"id":7,"vertexType":"COLUMN","vertexId":"default.s
 rc2.value2"}]}
 PREHOOK: query: insert into table dest2 select * from src1 JOIN src2 ON src1.key = src2.key2
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src1
 PREHOOK: Input: default@src2
 PREHOOK: Output: default@dest2
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "efeaddd0d36105b1013b414627850dc2",
-  "queryText": "insert into table dest2 select * from src1 JOIN src2 ON src1.key = src2.key2",
-  "edges": [
-    {
-      "sources": [
-        4
-      ],
-      "targets": [
-        0
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        5
-      ],
-      "targets": [
-        1
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        6
-      ],
-      "targets": [
-        2
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        7
-      ],
-      "targets": [
-        3
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        4,
-        6
-      ],
-      "targets": [
-        0,
-        1,
-        2,
-        3
-      ],
-      "expression": "(src1.key = src2.key2)",
-      "edgeType": "PREDICATE"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest2.key"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest2.value"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest2.key2"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest2.value2"
-    },
-    {
-      "id": 4,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.key"
-    },
-    {
-      "id": 5,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.value"
-    },
-    {
-      "id": 6,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src2.key2"
-    },
-    {
-      "id": 7,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src2.value2"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"efeaddd0d36105b1013b414627850dc2","queryText":"insert into table dest2 select * from src1 JOIN src2 ON src1.key = src2.key2","edges":[{"sources":[4],"targets":[0],"edgeType":"PROJECTION"},{"sources":[5],"targets":[1],"edgeType":"PROJECTION"},{"sources":[6],"targets":[2],"edgeType":"PROJECTION"},{"sources":[7],"targets":[3],"edgeType":"PROJECTION"},{"sources":[4,6],"targets":[0,1,2,3],"expression":"(src1.key = src2.key2)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.dest2.key"},{"id":1,"vertexType":"COLUMN","vertexId":"default.dest2.value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.dest2.key2"},{"id":3,"vertexType":"COLUMN","vertexId":"default.dest2.value2"},{"id":4,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":5,"vertexType":"COLUMN","vertexId":"default.src1.value"},{"id":6,"vertexType":"COLUMN","vertexId":"default.src2.key2"},{"id":7,"vertexType":"COLUMN","vertexId":"default.src2.v
 alue2"}]}
 PREHOOK: query: insert into table dest2
   select * from src1 JOIN src2 ON length(src1.value) = length(src2.value2) + 1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src1
 PREHOOK: Input: default@src2
 PREHOOK: Output: default@dest2
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "e9450a56b3d103642e06bef0e4f0d482",
-  "queryText": "insert into table dest2\n  select * from src1 JOIN src2 ON length(src1.value) = length(src2.value2) + 1",
-  "edges": [
-    {
-      "sources": [
-        4
-      ],
-      "targets": [
-        0
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        5
-      ],
-      "targets": [
-        1
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        6
-      ],
-      "targets": [
-        2
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        7
-      ],
-      "targets": [
-        3
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        5,
-        7
-      ],
-      "targets": [
-        0,
-        1,
-        2,
-        3
-      ],
-      "expression": "(length(src1.value) = (length(src2.value2) + 1))",
-      "edgeType": "PREDICATE"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest2.key"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest2.value"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest2.key2"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest2.value2"
-    },
-    {
-      "id": 4,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.key"
-    },
-    {
-      "id": 5,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.value"
-    },
-    {
-      "id": 6,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src2.key2"
-    },
-    {
-      "id": 7,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src2.value2"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"e9450a56b3d103642e06bef0e4f0d482","queryText":"insert into table dest2\n  select * from src1 JOIN src2 ON length(src1.value) = length(src2.value2) + 1","edges":[{"sources":[4],"targets":[0],"edgeType":"PROJECTION"},{"sources":[5],"targets":[1],"edgeType":"PROJECTION"},{"sources":[6],"targets":[2],"edgeType":"PROJECTION"},{"sources":[7],"targets":[3],"edgeType":"PROJECTION"},{"sources":[5,7],"targets":[0,1,2,3],"expression":"(length(src1.value) = (length(src2.value2) + 1))","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.dest2.key"},{"id":1,"vertexType":"COLUMN","vertexId":"default.dest2.value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.dest2.key2"},{"id":3,"vertexType":"COLUMN","vertexId":"default.dest2.value2"},{"id":4,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":5,"vertexType":"COLUMN","vertexId":"default.src1.value"},{"id":6,"vertexType":"COLUMN","vertexId":"default.src2.key2"},{"i
 d":7,"vertexType":"COLUMN","vertexId":"default.src2.value2"}]}
 PREHOOK: query: select * from src1 where length(key) > 2
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src1
 #### A masked pattern was here ####
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "4028c94d222d5dd221f651d414386972",
-  "queryText": "select * from src1 where length(key) > 2",
-  "edges": [
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        3
-      ],
-      "targets": [
-        1
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0,
-        1
-      ],
-      "expression": "(length(src1.key) > 2)",
-      "edgeType": "PREDICATE"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "src1.key"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "src1.value"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.key"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.value"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"4028c94d222d5dd221f651d414386972","queryText":"select * from src1 where length(key) > 2","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"},{"sources":[2],"targets":[0,1],"expression":"(length(src1.key) > 2)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"src1.key"},{"id":1,"vertexType":"COLUMN","vertexId":"src1.value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"default.src1.value"}]}
 238	val_238
 311	val_311
 255	val_255
@@ -1694,66 +503,7 @@ PREHOOK: query: select * from src1 where length(key) > 2 and value > 'a'
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src1
 #### A masked pattern was here ####
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "5727531f7743cfcd60d634d8c835515f",
-  "queryText": "select * from src1 where length(key) > 2 and value > 'a'",
-  "edges": [
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        3
-      ],
-      "targets": [
-        1
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        2,
-        3
-      ],
-      "targets": [
-        0,
-        1
-      ],
-      "expression": "((length(src1.key) > 2) and (src1.value > 'a'))",
-      "edgeType": "PREDICATE"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "src1.key"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "src1.value"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.key"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.value"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"5727531f7743cfcd60d634d8c835515f","queryText":"select * from src1 where length(key) > 2 and value > 'a'","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"},{"sources":[2,3],"targets":[0,1],"expression":"((length(src1.key) > 2) and (src1.value > 'a'))","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"src1.key"},{"id":1,"vertexType":"COLUMN","vertexId":"src1.value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"default.src1.value"}]}
 238	val_238
 311	val_311
 255	val_255
@@ -1773,238 +523,14 @@ PREHOOK: Input: default@src1
 PREHOOK: Input: default@src2
 PREHOOK: Output: database:default
 PREHOOK: Output: default@dest3
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "a2c4e9a3ec678039814f5d84b1e38ce4",
-  "queryText": "create table dest3 as\n  select * from src1 JOIN src2 ON src1.key = src2.key2 WHERE length(key) > 1",
-  "edges": [
-    {
-      "sources": [
-        4
-      ],
-      "targets": [
-        0
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        5
-      ],
-      "targets": [
-        1
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        6
-      ],
-      "targets": [
-        2
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        7
-      ],
-      "targets": [
-        3
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        4
-      ],
-      "targets": [
-        0,
-        1,
-        2,
-        3
-      ],
-      "expression": "(length(src1.key) > 1)",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [
-        4,
-        6
-      ],
-      "targets": [
-        0,
-        1,
-        2,
-        3
-      ],
-      "expression": "(src1.key = src2.key2)",
-      "edgeType": "PREDICATE"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest3.key"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest3.value"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest3.key2"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest3.value2"
-    },
-    {
-      "id": 4,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.key"
-    },
-    {
-      "id": 5,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.value"
-    },
-    {
-      "id": 6,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src2.key2"
-    },
-    {
-      "id": 7,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src2.value2"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"a2c4e9a3ec678039814f5d84b1e38ce4","queryText":"create table dest3 as\n  select * from src1 JOIN src2 ON src1.key = src2.key2 WHERE length(key) > 1","edges":[{"sources":[4],"targets":[0],"edgeType":"PROJECTION"},{"sources":[5],"targets":[1],"edgeType":"PROJECTION"},{"sources":[6],"targets":[2],"edgeType":"PROJECTION"},{"sources":[7],"targets":[3],"edgeType":"PROJECTION"},{"sources":[4],"targets":[0,1,2,3],"expression":"(length(src1.key) > 1)","edgeType":"PREDICATE"},{"sources":[4,6],"targets":[0,1,2,3],"expression":"(src1.key = src2.key2)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.dest3.key"},{"id":1,"vertexType":"COLUMN","vertexId":"default.dest3.value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.dest3.key2"},{"id":3,"vertexType":"COLUMN","vertexId":"default.dest3.value2"},{"id":4,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":5,"vertexType":"COLUMN","vertexId":"default.src1.value"
 },{"id":6,"vertexType":"COLUMN","vertexId":"default.src2.key2"},{"id":7,"vertexType":"COLUMN","vertexId":"default.src2.value2"}]}
 PREHOOK: query: insert overwrite table dest2
   select * from src1 JOIN src2 ON src1.key = src2.key2 WHERE length(key) > 3
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src1
 PREHOOK: Input: default@src2
 PREHOOK: Output: default@dest2
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "76d84512204ddc576ad4d93f252e4358",
-  "queryText": "insert overwrite table dest2\n  select * from src1 JOIN src2 ON src1.key = src2.key2 WHERE length(key) > 3",
-  "edges": [
-    {
-      "sources": [
-        4
-      ],
-      "targets": [
-        0
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        5
-      ],
-      "targets": [
-        1
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        6
-      ],
-      "targets": [
-        2
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        7
-      ],
-      "targets": [
-        3
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        4
-      ],
-      "targets": [
-        0,
-        1,
-        2,
-        3
-      ],
-      "expression": "(length(src1.key) > 3)",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [
-        4,
-        6
-      ],
-      "targets": [
-        0,
-        1,
-        2,
-        3
-      ],
-      "expression": "(src1.key = src2.key2)",
-      "edgeType": "PREDICATE"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest2.key"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest2.value"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest2.key2"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest2.value2"
-    },
-    {
-      "id": 4,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.key"
-    },
-    {
-      "id": 5,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.value"
-    },
-    {
-      "id": 6,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src2.key2"
-    },
-    {
-      "id": 7,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src2.value2"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"76d84512204ddc576ad4d93f252e4358","queryText":"insert overwrite table dest2\n  select * from src1 JOIN src2 ON src1.key = src2.key2 WHERE length(key) > 3","edges":[{"sources":[4],"targets":[0],"edgeType":"PROJECTION"},{"sources":[5],"targets":[1],"edgeType":"PROJECTION"},{"sources":[6],"targets":[2],"edgeType":"PROJECTION"},{"sources":[7],"targets":[3],"edgeType":"PROJECTION"},{"sources":[4],"targets":[0,1,2,3],"expression":"(length(src1.key) > 3)","edgeType":"PREDICATE"},{"sources":[4,6],"targets":[0,1,2,3],"expression":"(src1.key = src2.key2)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.dest2.key"},{"id":1,"vertexType":"COLUMN","vertexId":"default.dest2.value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.dest2.key2"},{"id":3,"vertexType":"COLUMN","vertexId":"default.dest2.value2"},{"id":4,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":5,"vertexType":"COLUMN","vertexId":"default.src1
 .value"},{"id":6,"vertexType":"COLUMN","vertexId":"default.src2.key2"},{"id":7,"vertexType":"COLUMN","vertexId":"default.src2.value2"}]}
 PREHOOK: query: drop table if exists dest_l1
 PREHOOK: type: DROPTABLE
 PREHOOK: query: CREATE TABLE dest_l1(key INT, value STRING) STORED AS TEXTFILE
@@ -2026,73 +552,7 @@ PREHOOK: type: QUERY
 PREHOOK: Input: default@src
 PREHOOK: Input: default@src1
 PREHOOK: Output: default@dest_l1
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "60b589744e2527dd235a6c8168d6a653",
-  "queryText": "INSERT OVERWRITE TABLE dest_l1\nSELECT j.*\nFROM (SELECT t1.key, p1.value\n      FROM src1 t1\n      LEFT OUTER JOIN src p1\n      ON (t1.key = p1.key)\n      UNION ALL\n      SELECT t2.key, p2.value\n      FROM src1 t2\n      LEFT OUTER JOIN src p2\n      ON (t2.key = p2.key)) j",
-  "edges": [
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0
-      ],
-      "expression": "UDFToInteger(j.key)",
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        3
-      ],
-      "targets": [
-        1
-      ],
-      "expression": "j.value",
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        4,
-        2
-      ],
-      "targets": [
-        0,
-        1
-      ],
-      "expression": "(p1.key = t1.key)",
-      "edgeType": "PREDICATE"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_l1.key"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_l1.value"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.key"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src.value"
-    },
-    {
-      "id": 4,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src.key"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"60b589744e2527dd235a6c8168d6a653","queryText":"INSERT OVERWRITE TABLE dest_l1\nSELECT j.*\nFROM (SELECT t1.key, p1.value\n      FROM src1 t1\n      LEFT OUTER JOIN src p1\n      ON (t1.key = p1.key)\n      UNION ALL\n      SELECT t2.key, p2.value\n      FROM src1 t2\n      LEFT OUTER JOIN src p2\n      ON (t2.key = p2.key)) j","edges":[{"sources":[2],"targets":[0],"expression":"UDFToInteger(j.key)","edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"expression":"j.value","edgeType":"PROJECTION"},{"sources":[4,2],"targets":[0,1],"expression":"(p1.key = t1.key)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.dest_l1.key"},{"id":1,"vertexType":"COLUMN","vertexId":"default.dest_l1.value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"default.src.value"},{"id":4,"vertexType":"COLUMN","vertexId":"default.src.key"}]}
 PREHOOK: query: drop table if exists emp
 PREHOOK: type: DROPTABLE
 PREHOOK: query: drop table if exists dept
@@ -2133,185 +593,7 @@ PREHOOK: Input: default@dept
 PREHOOK: Input: default@emp
 PREHOOK: Input: default@project
 PREHOOK: Output: default@tgt
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "f59797e0422d2e51515063374dfac361",
-  "queryText": "INSERT INTO TABLE tgt\nSELECT emd.dept_name, emd.name, emd.emp_id, emd.mgr_id, p.project_id, p.project_name\nFROM (\n  SELECT d.dept_name, em.name, em.emp_id, em.mgr_id, em.dept_id\n  FROM (\n    SELECT e.name, e.dept_id, e.emp_id emp_id, m.emp_id mgr_id\n    FROM emp e JOIN emp m ON e.emp_id = m.emp_id\n    ) em\n  JOIN dept d ON d.dept_id = em.dept_id\n  ) emd JOIN project p ON emd.dept_id = p.project_id",
-  "edges": [
-    {
-      "sources": [
-        6
-      ],
-      "targets": [
-        0
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        7
-      ],
-      "targets": [
-        1
-      ],
-      "expression": "emd.name",
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        8
-      ],
-      "targets": [
-        2
-      ],
-      "expression": "emd.emp_id",
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        8
-      ],
-      "targets": [
-        3
-      ],
-      "expression": "emd.mgr_id",
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        9
-      ],
-      "targets": [
-        4
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        10
-      ],
-      "targets": [
-        5
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        8
-      ],
-      "targets": [
-        0,
-        1,
-        2,
-        3,
-        4,
-        5
-      ],
-      "expression": "(e.emp_id = m.emp_id)",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [
-        11,
-        12
-      ],
-      "targets": [
-        0,
-        1,
-        2,
-        3,
-        4,
-        5
-      ],
-      "expression": "(em._col1 = d.dept_id)",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [
-        11,
-        9
-      ],
-      "targets": [
-        0,
-        1,
-        2,
-        3,
-        4,
-        5
-      ],
-      "expression": "(emd._col4 = p.project_id)",
-      "edgeType": "PREDICATE"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "default.tgt.dept_name"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "default.tgt.name"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.tgt.emp_id"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.tgt.mgr_id"
-    },
-    {
-      "id": 4,
-      "vertexType": "COLUMN",
-      "vertexId": "default.tgt.proj_id"
-    },
-    {
-      "id": 5,
-      "vertexType": "COLUMN",
-      "vertexId": "default.tgt.proj_name"
-    },
-    {
-      "id": 6,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dept.dept_name"
-    },
-    {
-      "id": 7,
-      "vertexType": "COLUMN",
-      "vertexId": "default.emp.name"
-    },
-    {
-      "id": 8,
-      "vertexType": "COLUMN",
-      "vertexId": "default.emp.emp_id"
-    },
-    {
-      "id": 9,
-      "vertexType": "COLUMN",
-      "vertexId": "default.project.project_id"
-    },
-    {
-      "id": 10,
-      "vertexType": "COLUMN",
-      "vertexId": "default.project.project_name"
-    },
-    {
-      "id": 11,
-      "vertexType": "COLUMN",
-      "vertexId": "default.emp.dept_id"
-    },
-    {
-      "id": 12,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dept.dept_id"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"f59797e0422d2e51515063374dfac361","queryText":"INSERT INTO TABLE tgt\nSELECT emd.dept_name, emd.name, emd.emp_id, emd.mgr_id, p.project_id, p.project_name\nFROM (\n  SELECT d.dept_name, em.name, em.emp_id, em.mgr_id, em.dept_id\n  FROM (\n    SELECT e.name, e.dept_id, e.emp_id emp_id, m.emp_id mgr_id\n    FROM emp e JOIN emp m ON e.emp_id = m.emp_id\n    ) em\n  JOIN dept d ON d.dept_id = em.dept_id\n  ) emd JOIN project p ON emd.dept_id = p.project_id","edges":[{"sources":[6],"targets":[0],"edgeType":"PROJECTION"},{"sources":[7],"targets":[1],"expression":"emd.name","edgeType":"PROJECTION"},{"sources":[8],"targets":[2],"expression":"emd.emp_id","edgeType":"PROJECTION"},{"sources":[8],"targets":[3],"expression":"emd.mgr_id","edgeType":"PROJECTION"},{"sources":[9],"targets":[4],"edgeType":"PROJECTION"},{"sources":[10],"targets":[5],"edgeType":"PROJECTION"},{"sources":[8],"targets":[0,1,2,3,4,5],"expression":"(e.emp_id = m.emp_id)","edgeType":"PR
 EDICATE"},{"sources":[11,12],"targets":[0,1,2,3,4,5],"expression":"(em._col1 = d.dept_id)","edgeType":"PREDICATE"},{"sources":[11,9],"targets":[0,1,2,3,4,5],"expression":"(emd._col4 = p.project_id)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.tgt.dept_name"},{"id":1,"vertexType":"COLUMN","vertexId":"default.tgt.name"},{"id":2,"vertexType":"COLUMN","vertexId":"default.tgt.emp_id"},{"id":3,"vertexType":"COLUMN","vertexId":"default.tgt.mgr_id"},{"id":4,"vertexType":"COLUMN","vertexId":"default.tgt.proj_id"},{"id":5,"vertexType":"COLUMN","vertexId":"default.tgt.proj_name"},{"id":6,"vertexType":"COLUMN","vertexId":"default.dept.dept_name"},{"id":7,"vertexType":"COLUMN","vertexId":"default.emp.name"},{"id":8,"vertexType":"COLUMN","vertexId":"default.emp.emp_id"},{"id":9,"vertexType":"COLUMN","vertexId":"default.project.project_id"},{"id":10,"vertexType":"COLUMN","vertexId":"default.project.project_name"},{"id":11,"vertexType":"COLUMN","vertexId":
 "default.emp.dept_id"},{"id":12,"vertexType":"COLUMN","vertexId":"default.dept.dept_id"}]}
 PREHOOK: query: drop table if exists dest_l2
 PREHOOK: type: DROPTABLE
 PREHOOK: query: create table dest_l2 (id int, c1 tinyint, c2 int, c3 bigint) stored as textfile
@@ -2322,68 +604,7 @@ PREHOOK: query: insert into dest_l2 values(0, 1, 100, 10000)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@values__tmp__table__1
 PREHOOK: Output: default@dest_l2
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "e001334e3f8384806b0f25a7c303045f",
-  "queryText": "insert into dest_l2 values(0, 1, 100, 10000)",
-  "edges": [
-    {
-      "sources": [],
-      "targets": [
-        0
-      ],
-      "expression": "UDFToInteger(values__tmp__table__1.tmp_values_col1)",
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [],
-      "targets": [
-        1
-      ],
-      "expression": "UDFToByte(values__tmp__table__1.tmp_values_col2)",
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [],
-      "targets": [
-        2
-      ],
-      "expression": "UDFToInteger(values__tmp__table__1.tmp_values_col3)",
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [],
-      "targets": [
-        3
-      ],
-      "expression": "UDFToLong(values__tmp__table__1.tmp_values_col4)",
-      "edgeType": "PROJECTION"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_l2.id"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_l2.c1"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_l2.c2"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_l2.c3"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"e001334e3f8384806b0f25a7c303045f","queryText":"insert into dest_l2 values(0, 1, 100, 10000)","edges":[{"sources":[],"targets":[0],"expression":"UDFToInteger(values__tmp__table__1.tmp_values_col1)","edgeType":"PROJECTION"},{"sources":[],"targets":[1],"expression":"UDFToByte(values__tmp__table__1.tmp_values_col2)","edgeType":"PROJECTION"},{"sources":[],"targets":[2],"expression":"UDFToInteger(values__tmp__table__1.tmp_values_col3)","edgeType":"PROJECTION"},{"sources":[],"targets":[3],"expression":"UDFToLong(values__tmp__table__1.tmp_values_col4)","edgeType":"PROJECTION"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.dest_l2.id"},{"id":1,"vertexType":"COLUMN","vertexId":"default.dest_l2.c1"},{"id":2,"vertexType":"COLUMN","vertexId":"default.dest_l2.c2"},{"id":3,"vertexType":"COLUMN","vertexId":"default.dest_l2.c3"}]}
 PREHOOK: query: select * from (
   select c1 + c2 x from dest_l2
   union all
@@ -2391,48 +612,7 @@ PREHOOK: query: select * from (
 PREHOOK: type: QUERY
 PREHOOK: Input: default@dest_l2
 #### A masked pattern was here ####
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "a2c96a96be9d315ede966be5b45ef20e",
-  "queryText": "select * from (\n  select c1 + c2 x from dest_l2\n  union all\n  select sum(c3) y from (select c3 from dest_l2) v1) v2 order by x",
-  "edges": [
-    {
-      "sources": [
-        1,
-        2,
-        3
-      ],
-      "targets": [
-        0
-      ],
-      "expression": "v2.x",
-      "edgeType": "PROJECTION"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "v2.x"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_l2.c1"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_l2.c2"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_l2.c3"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"a2c96a96be9d315ede966be5b45ef20e","queryText":"select * from (\n  select c1 + c2 x from dest_l2\n  union all\n  select sum(c3) y from (select c3 from dest_l2) v1) v2 order by x","edges":[{"sources":[1,2,3],"targets":[0],"expression":"v2.x","edgeType":"PROJECTION"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"v2.x"},{"id":1,"vertexType":"COLUMN","vertexId":"default.dest_l2.c1"},{"id":2,"vertexType":"COLUMN","vertexId":"default.dest_l2.c2"},{"id":3,"vertexType":"COLUMN","vertexId":"default.dest_l2.c3"}]}
 101
 10000
 PREHOOK: query: drop table if exists dest_l3
@@ -2445,60 +625,7 @@ PREHOOK: query: insert into dest_l3 values(0, "s1", "s2", 15)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@values__tmp__table__2
 PREHOOK: Output: default@dest_l3
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "09df51ba6ba2d07f2304523ee505f094",
-  "queryText": "insert into dest_l3 values(0, \"s1\", \"s2\", 15)",
-  "edges": [
-    {
-      "sources": [],
-      "targets": [
-        0
-      ],
-      "expression": "UDFToInteger(values__tmp__table__2.tmp_values_col1)",
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [],
-      "targets": [
-        1,
-        2
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [],
-      "targets": [
-        3
-      ],
-      "expression": "UDFToInteger(values__tmp__table__2.tmp_values_col4)",
-      "edgeType": "PROJECTION"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_l3.id"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_l3.c1"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_l3.c2"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_l3.c3"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"09df51ba6ba2d07f2304523ee505f094","queryText":"insert into dest_l3 values(0, \"s1\", \"s2\", 15)","edges":[{"sources":[],"targets":[0],"expression":"UDFToInteger(values__tmp__table__2.tmp_values_col1)","edgeType":"PROJECTION"},{"sources":[],"targets":[1,2],"edgeType":"PROJECTION"},{"sources":[],"targets":[3],"expression":"UDFToInteger(values__tmp__table__2.tmp_values_col4)","edgeType":"PROJECTION"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.dest_l3.id"},{"id":1,"vertexType":"COLUMN","vertexId":"default.dest_l3.c1"},{"id":2,"vertexType":"COLUMN","vertexId":"default.dest_l3.c2"},{"id":3,"vertexType":"COLUMN","vertexId":"default.dest_l3.c3"}]}
 PREHOOK: query: select sum(a.c1) over (partition by a.c1 order by a.id)
 from dest_l2 a
 where a.c2 != 10
@@ -2507,68 +634,7 @@ having count(a.c2) > 0
 PREHOOK: type: QUERY
 PREHOOK: Input: default@dest_l2
 #### A masked pattern was here ####
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "0ae7aa4a0cbd1283210fa79e8a19104a",
-  "queryText": "select sum(a.c1) over (partition by a.c1 order by a.id)\nfrom dest_l2 a\nwhere a.c2 != 10\ngroup by a.c1, a.c2, a.id\nhaving count(a.c2) > 0",
-  "edges": [
-    {
-      "sources": [
-        1,
-        2,
-        3
-      ],
-      "targets": [
-        0
-      ],
-      "expression": "(tok_function sum (. (tok_table_or_col $hdt$_0) $f0) (tok_windowspec (tok_partitioningspec (tok_distributeby (. (tok_table_or_col $hdt$_0) $f0)) (tok_orderby (tok_tabsortcolnameasc (. (tok_table_or_col $hdt$_0) $f2)))) (tok_windowvalues (preceding 2147483647) current)))",
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0
-      ],
-      "expression": "(a.c2 <> 10)",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0
-      ],
-      "expression": "(count(default.dest_l2.c2) > 0)",
-      "edgeType": "PREDICATE"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "c0"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_l2.c1"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_l2.c2"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_l2.id"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"0ae7aa4a0cbd1283210fa79e8a19104a","queryText":"select sum(a.c1) over (partition by a.c1 order by a.id)\nfrom dest_l2 a\nwhere a.c2 != 10\ngroup by a.c1, a.c2, a.id\nhaving count(a.c2) > 0","edges":[{"sources":[1,2,3],"targets":[0],"expression":"(tok_function sum (. (tok_table_or_col $hdt$_0) $f0) (tok_windowspec (tok_partitioningspec (tok_distributeby (. (tok_table_or_col $hdt$_0) $f0)) (tok_orderby (tok_tabsortcolnameasc (. (tok_table_or_col $hdt$_0) $f2)))) (tok_windowvalues (preceding 2147483647) current)))","edgeType":"PROJECTION"},{"sources":[2],"targets":[0],"expression":"(a.c2 <> 10)","edgeType":"PREDICATE"},{"sources":[2],"targets":[0],"expression":"(count(default.dest_l2.c2) > 0)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"c0"},{"id":1,"vertexType":"COLUMN","vertexId":"default.dest_l2.c1"},{"id":2,"vertexType":"COLUMN","vertexId":"default.dest_l2.c2"},{"id":3,"vertexType":"COLUMN","vertexId":"default
 .dest_l2.id"}]}
 1
 PREHOOK: query: select sum(a.c1), count(b.c1), b.c2, b.c3
 from dest_l2 a join dest_l3 b on (a.id = b.id)
@@ -2580,150 +646,7 @@ PREHOOK: type: QUERY
 PREHOOK: Input: default@dest_l2
 PREHOOK: Input: default@dest_l3
 #### A masked pattern was here ####
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "01879c619517509d9f5b6ead998bb4bb",
-  "queryText": "select sum(a.c1), count(b.c1), b.c2, b.c3\nfrom dest_l2 a join dest_l3 b on (a.id = b.id)\nwhere a.c2 != 10 and b.c3 > 0\ngroup by a.c1, a.c2, a.id, b.c1, b.c2, b.c3\nhaving count(a.c2) > 0\norder by b.c3 limit 5",
-  "edges": [
-    {
-      "sources": [
-        4
-      ],
-      "targets": [
-        0
-      ],
-      "expression": "sum(default.dest_l2.c1)",
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        5
-      ],
-      "targets": [
-        1
-      ],
-      "expression": "count(default.dest_l3.c1)",
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        6
-      ],
-      "targets": [
-        2
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        7
-      ],
-      "targets": [
-        3
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        8,
-        7
-      ],
-      "targets": [
-        0,
-        1,
-        2,
-        3
-      ],
-      "expression": "((a.c2 <> 10) and (b.c3 > 0))",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [
-        8
-      ],
-      "targets": [
-        0,
-        1,
-        2,
-        3
-      ],
-      "expression": "(count(default.dest_l2.c2) > 0)",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [
-        9,
-        10
-      ],
-      "targets": [
-        0,
-        1,
-        2,
-        3
-      ],
-      "expression": "(a.id = b.id)",
-      "edgeType": "PREDICATE"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "_c0"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "_c1"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "b.c2"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "b.c3"
-    },
-    {
-      "id": 4,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_l2.c1"
-    },
-    {
-      "id": 5,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_l3.c1"
-    },
-    {
-      "id": 6,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_l3.c2"
-    },
-    {
-      "id": 7,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_l3.c3"
-    },
-    {
-      "id": 8,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_l2.c2"
-    },
-    {
-      "id": 9,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_l2.id"
-    },
-    {
-      "id": 10,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_l3.id"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"01879c619517509d9f5b6ead998bb4bb","queryText":"select sum(a.c1), count(b.c1), b.c2, b.c3\nfrom dest_l2 a join dest_l3 b on (a.id = b.id)\nwhere a.c2 != 10 and b.c3 > 0\ngroup by a.c1, a.c2, a.id, b.c1, b.c2, b.c3\nhaving count(a.c2) > 0\norder by b.c3 limit 5","edges":[{"sources":[4],"targets":[0],"expression":"sum(default.dest_l2.c1)","edgeType":"PROJECTION"},{"sources":[5],"targets":[1],"expression":"count(default.dest_l3.c1)","edgeType":"PROJECTION"},{"sources":[6],"targets":[2],"edgeType":"PROJECTION"},{"sources":[7],"targets":[3],"edgeType":"PROJECTION"},{"sources":[8,7],"targets":[0,1,2,3],"expression":"((a.c2 <> 10) and (b.c3 > 0))","edgeType":"PREDICATE"},{"sources":[8],"targets":[0,1,2,3],"expression":"(count(default.dest_l2.c2) > 0)","edgeType":"PREDICATE"},{"sources":[9,10],"targets":[0,1,2,3],"expression":"(a.id = b.id)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"_c0"},{"id":1,"vertexType":"COLUMN
 ","vertexId":"_c1"},{"id":2,"vertexType":"COLUMN","vertexId":"b.c2"},{"id":3,"vertexType":"COLUMN","vertexId":"b.c3"},{"id":4,"vertexType":"COLUMN","vertexId":"default.dest_l2.c1"},{"id":5,"vertexType":"COLUMN","vertexId":"default.dest_l3.c1"},{"id":6,"vertexType":"COLUMN","vertexId":"default.dest_l3.c2"},{"id":7,"vertexType":"COLUMN","vertexId":"default.dest_l3.c3"},{"id":8,"vertexType":"COLUMN","vertexId":"default.dest_l2.c2"},{"id":9,"vertexType":"COLUMN","vertexId":"default.dest_l2.id"},{"id":10,"vertexType":"COLUMN","vertexId":"default.dest_l3.id"}]}
 1	1	s2	15
 PREHOOK: query: drop table if exists t
 PREHOOK: type: DROPTABLE
@@ -2736,93 +659,7 @@ PREHOOK: Input: default@dest_l2
 PREHOOK: Input: default@dest_l3
 PREHOOK: Output: database:default
 PREHOOK: Output: default@t
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "0d2f15b494111ffe236d5be42a76fa28",
-  "queryText": "create table t as\nselect distinct a.c2, a.c3 from dest_l2 a\ninner join dest_l3 b on (a.id = b.id)\nwhere a.id > 0 and b.c3 = 15",
-  "edges": [
-    {
-      "sources": [
-        2
-      ],
-      "targets": [
-        0
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        3
-      ],
-      "targets": [
-        1
-      ],
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        4,
-        5
-      ],
-      "targets": [
-        0,
-        1
-      ],
-      "expression": "((a.id > 0) and (b.c3 = 15))",
-      "edgeType": "PREDICATE"
-    },
-    {
-      "sources": [
-        4,
-        6
-      ],
-      "targets": [
-        0,
-        1
-      ],
-      "expression": "(a.id = b.id)",
-      "edgeType": "PREDICATE"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "default.t.c2"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "default.t.c3"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_l2.c2"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_l2.c3"
-    },
-    {
-      "id": 4,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_l2.id"
-    },
-    {
-      "id": 5,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_l3.c3"
-    },
-    {
-      "id": 6,
-      "vertexType": "COLUMN",
-      "vertexId": "default.dest_l3.id"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"0d2f15b494111ffe236d5be42a76fa28","queryText":"create table t as\nselect distinct a.c2, a.c3 from dest_l2 a\ninner join dest_l3 b on (a.id = b.id)\nwhere a.id > 0 and b.c3 = 15","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"},{"sources":[4,5],"targets":[0,1],"expression":"((a.id > 0) and (b.c3 = 15))","edgeType":"PREDICATE"},{"sources":[4,6],"targets":[0,1],"expression":"(a.id = b.id)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.t.c2"},{"id":1,"vertexType":"COLUMN","vertexId":"default.t.c3"},{"id":2,"vertexType":"COLUMN","vertexId":"default.dest_l2.c2"},{"id":3,"vertexType":"COLUMN","vertexId":"default.dest_l2.c3"},{"id":4,"vertexType":"COLUMN","vertexId":"default.dest_l2.id"},{"id":5,"vertexType":"COLUMN","vertexId":"default.dest_l3.c3"},{"id":6,"vertexType":"COLUMN","vertexId":"default.dest_l3.id"}]}
 PREHOOK: query: SELECT substr(src1.key,1,1), count(DISTINCT substr(src1.value,5)),
 concat(substr(src1.key,1,1),sum(substr(src1.value,5)))
 from src1
@@ -2830,72 +667,7 @@ GROUP BY substr(src1.key,1,1)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src1
 #### A masked pattern was here ####
-POSTHOOK: LINEAGE: {
-  "version": "1.0",
-  "engine": "mr",
-  "hash": "5b1022708124ee2b80f9e2e8a0dcb15c",
-  "queryText": "SELECT substr(src1.key,1,1), count(DISTINCT substr(src1.value,5)),\nconcat(substr(src1.key,1,1),sum(substr(src1.value,5)))\nfrom src1\nGROUP BY substr(src1.key,1,1)",
-  "edges": [
-    {
-      "sources": [
-        3
-      ],
-      "targets": [
-        0
-      ],
-      "expression": "substr(src1.key, 1, 1)",
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        4
-      ],
-      "targets": [
-        1
-      ],
-      "expression": "count(DISTINCT substr(src1.value, 5))",
-      "edgeType": "PROJECTION"
-    },
-    {
-      "sources": [
-        3,
-        4
-      ],
-      "targets": [
-        2
-      ],
-      "expression": "concat(substr(src1.key, 1, 1), sum(substr(src1.value, 5)))",
-      "edgeType": "PROJECTION"
-    }
-  ],
-  "vertices": [
-    {
-      "id": 0,
-      "vertexType": "COLUMN",
-      "vertexId": "c0"
-    },
-    {
-      "id": 1,
-      "vertexType": "COLUMN",
-      "vertexId": "c1"
-    },
-    {
-      "id": 2,
-      "vertexType": "COLUMN",
-      "vertexId": "c2"
-    },
-    {
-      "id": 3,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.key"
-    },
-    {
-      "id": 4,
-      "vertexType": "COLUMN",
-      "vertexId": "default.src1.value"
-    }
-  ]
-}
+{"version":"1.0","engine":"mr","hash":"5b1022708124ee2b80f9e2e8a0dcb15c","queryText":"SELECT substr(src1.key,1,1), count(DISTINCT substr(src1.value,5)),\nconcat(substr(src1.key,1,1),sum(substr(src1.value,5)))\nfrom src1\nGROUP BY substr(src1.key,1,1)","edges":[{"sources":[3],"targets":[0],"expression":"substr(src1.key, 1, 1)","edgeType":"PROJECTION"},{"sources":[4],"targets":[1],"expression":"count(DISTINCT substr(src1.value, 5))","edgeType":"PROJECTION"},{"sources":[3,4],"targets":[2],"expression":"concat(substr(src1.key, 1, 1), sum(substr(src1.value, 5)))","edgeType":"PROJECTION"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"c0"},{"id":1,"vertexType":"COLUMN","vertexId":"c1"},{"id":2,"vertexType":"COLUMN","vertexId":"c2"},{"id":3,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":4,"vertexType":"COLUMN","vertexId":"default.src1.value"}]}
 	7	1543.0
 1	3	1296.0
 2	6	21257.0